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: 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/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index f2ea3f8aab4c..a6d4fc7e5c2e 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -28,7 +28,9 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.5.0 (latest release)" + - "1.5.2 (latest release)" + - "1.5.1" + - "1.5.0" - "1.4.3" - "1.4.2" - "1.4.1" 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/.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: 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 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/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/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/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/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/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/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/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/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++) { 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); + } +} 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..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 @@ -34,11 +34,16 @@ 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; +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"}) @@ -75,7 +80,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 +108,7 @@ public static void beforeClass() { ImmutableMap.of()); } - @AfterClass + @AfterAll public static void afterClass() { AwsIntegTestUtil.cleanGlueCatalog(glue, namespaces); AwsIntegTestUtil.cleanS3Bucket(s3, testBucketName, testPathPrefix); @@ -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/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..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 @@ -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()) + .containsExactlyInAnyOrderElementsOf(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,16 @@ 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); + String description = "test description"; + updateTableDescription(namespace, tableName, description); 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 +199,26 @@ 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()); + 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 @@ -229,10 +231,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 +250,7 @@ public void testRenameTableFailsToCreateNewTable() { .databaseName(namespace) .tableInput(TableInput.builder().name(newTableName).build()) .build()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalog.renameTable( TableIdentifier.of(namespace, tableName), @@ -258,10 +260,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 +279,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 +287,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 +301,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 +314,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 +322,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 +354,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 +368,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 +403,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 +438,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 +511,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 +543,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 +566,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 +585,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 +623,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..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 @@ -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,30 @@ 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 + 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 @@ -326,9 +336,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 +350,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 +418,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 +441,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 +460,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 +477,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/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/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(); } /** 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)) 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..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 @@ -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; @@ -91,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. * @@ -108,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"; @@ -788,10 +797,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 +843,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/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/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/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/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/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/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); 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)); } } 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/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 .*"; diff --git a/build.gradle b/build.gradle index c21813a4d4ae..314393dfd902 100644 --- a/build.gradle +++ b/build.gradle @@ -36,10 +36,10 @@ 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' + 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' } @@ -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) @@ -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') @@ -581,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 @@ -911,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 @@ -963,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/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(); } 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/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/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/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/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/ClientPoolImpl.java b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java index e8ab57fed3e2..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; @@ -34,16 +35,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,21 +70,25 @@ 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); + 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 action.run(client); } throw exc; - } finally { release(client); } @@ -144,6 +162,11 @@ private void release(C client) { } } + @VisibleForTesting + Deque clients() { + return clients; + } + public int poolSize() { return poolSize; } 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/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); 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/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/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/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 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/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/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/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index dc7352743e88..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; @@ -86,7 +87,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 +138,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(); } @@ -686,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/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/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 077c33321be4..c9bd2b78a628 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,11 @@ 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"; + + static final String RETRYABLE_STATUS_CODES = "retryable_status_codes"; enum SchemaVersion { V0, @@ -54,7 +59,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 +101,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 @@ -308,7 +340,7 @@ enum SchemaVersion { + TABLE_NAMESPACE + " = ? OR " + TABLE_NAMESPACE - + " LIKE ? ESCAPE '\\')" + + " LIKE ? ESCAPE '!')" + " LIMIT 1"; static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT " @@ -399,7 +431,7 @@ enum SchemaVersion { + NAMESPACE_NAME + " = ? OR " + NAMESPACE_NAME - + " LIKE ? ESCAPE '\\' " + + " LIKE ? ESCAPE '!' " + " ) "; static final String INSERT_NAMESPACE_PROPERTIES_SQL = "INSERT INTO " @@ -504,7 +536,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 +547,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(); } @@ -757,7 +788,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; } 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/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/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index 936663095492..e83ee650cf99 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,13 @@ 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.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; 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; @@ -39,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; @@ -79,31 +83,29 @@ 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; private HTTPClient( String uri, + HttpHost proxy, + CredentialsProvider proxyCredsProvider, 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) { @@ -120,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(); } @@ -448,6 +458,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); } @@ -457,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; @@ -468,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; @@ -493,7 +559,20 @@ public HTTPClient build() { interceptor = loadInterceptorDynamically(SIGV4_REQUEST_INTERCEPTOR_IMPL, properties); } - return new HTTPClient(uri, baseHeaders, mapper, interceptor, 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, + properties, + configureConnectionManager(properties)); } } 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/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; } 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/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 96aa14b128da..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; @@ -114,6 +115,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 +138,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 @@ -217,15 +220,27 @@ 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( - 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.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); @@ -278,14 +293,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 +522,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 @@ -558,7 +595,7 @@ public boolean updateNamespaceMetadata( return !response.updated().isEmpty(); } - private ScheduledExecutorService tokenRefreshExecutor() { + private ScheduledExecutorService tokenRefreshExecutor(String catalogName) { if (!keepTokenRefreshed) { return null; } @@ -566,7 +603,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); } } } @@ -602,7 +639,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) { @@ -930,7 +967,7 @@ private Pair> newSession( () -> AuthSession.fromAccessToken( client, - tokenRefreshExecutor(), + tokenRefreshExecutor(name()), credentials.get(OAuth2Properties.TOKEN), expiresAtMillis(properties), parent)); @@ -943,7 +980,7 @@ private Pair> newSession( () -> AuthSession.fromCredential( client, - tokenRefreshExecutor(), + tokenRefreshExecutor(name()), credentials.get(OAuth2Properties.CREDENTIAL), parent)); } @@ -956,7 +993,7 @@ private Pair> newSession( () -> AuthSession.fromTokenExchange( client, - tokenRefreshExecutor(), + tokenRefreshExecutor(name()), credentials.get(tokenType), tokenType, parent)); @@ -1048,14 +1085,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/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) { 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/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/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/TestClientPoolImpl.java b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java new file mode 100644 index 000000000000..3a6666bac38b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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)) { + // 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(); + } + } + + @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; + + MockClient() {} + + MockClient(int retryableFailures) { + this.retryableFailures = retryableFailures; + } + + 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 new MockClient(reconnectionAttempts); + } + + @Override + protected void close(MockClient client) { + client.close(); + } + + int reconnectionAttempts() { + return reconnectionAttempts; + } + } +} 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/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"); + } } 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/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/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); 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 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/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/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))); 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..90492b51097a 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; @@ -30,7 +31,10 @@ 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.sql.SQLNonTransientConnectionException; import java.util.List; import java.util.Map; import java.util.Set; @@ -41,6 +45,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 +56,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 +85,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 { @@ -161,6 +169,100 @@ 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 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(); @@ -234,6 +336,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 @@ -970,6 +1083,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); @@ -979,7 +1132,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); @@ -1000,7 +1195,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( @@ -1016,8 +1213,42 @@ 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(); } } + + 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; + } } 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); + } + } } 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/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/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/TestHTTPClient.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java index e596df43e6f5..ffd10dff932c 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java @@ -31,13 +31,21 @@ 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.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; @@ -47,9 +55,13 @@ 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.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 @@ -121,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"); @@ -133,6 +234,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; 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/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/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 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); + } } 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/data/src/test/java/org/apache/iceberg/data/DataTest.java b/data/src/test/java/org/apache/iceberg/data/DataTest.java index 7e32da4c6edf..638a344cd2bc 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,15 +31,14 @@ 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 { 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()), @@ -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/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/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/doap.rdf b/doap.rdf index d3ef2e34497c..7f49fc42f793 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.5.0 - 2024-03-11 - 1.5.0 + 1.5.2 + 2024-05-09 + 1.5.2 diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 2bd6636670ee..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 @@ -578,11 +579,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 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/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index ef1e602c8212..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.16/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: 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. 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 = 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/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. diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index a803f4585e41..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 @@ -60,7 +61,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 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.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/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/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/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/data/TestFlinkParquetReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java deleted file mode 100644 index 30a2a7bb51ce..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 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.Assert; -import org.junit.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 = temp.newFile(); - Assert.assertTrue(testFile.delete()); - - 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(); - Assert.assertTrue("Should have at least one row", 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()); - } - } - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); - - 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) { - Assert.assertTrue("Should have expected number of rows", rows.hasNext()); - TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); - } - Assert.assertFalse("Should not have extra rows", rows.hasNext()); - } - } - - @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/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.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/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/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/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 0655cf87a996..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 @@ -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); } } @@ -500,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/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/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/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/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/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/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/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/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/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/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/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/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/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..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,8 @@ 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; @@ -43,16 +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.junit.Assert; -import org.junit.Test; +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 = @@ -61,8 +162,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 +191,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 +220,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.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/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.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/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/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/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/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.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.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/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7d991ee603c9..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 @@ -18,10 +18,13 @@ */ 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; -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; @@ -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; @@ -54,10 +61,14 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; 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(); @@ -81,12 +92,17 @@ 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()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) .flinkConfig(config); } @@ -103,6 +119,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 + RecordCounterToWait.waitForCondition(); + CompletableFuture savepoint = + miniClusterResource + .getClusterClient() + .stopWithSavepoint( + jobId, + false, + TEMPORARY_FOLDER.newFolder().toPath().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 { testBoundedIcebergSource(FailoverType.TM); @@ -125,35 +190,17 @@ 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, 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)); @@ -222,6 +269,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 = + 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(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + } + // ------------------------------------------------------------------------ // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ @@ -263,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 f7dc931c506c..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 @@ -88,16 +97,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.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/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.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.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); } } } 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/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/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/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 0655cf87a996..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 @@ -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); } } @@ -500,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/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/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/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/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/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/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/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/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/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/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/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/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..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,8 @@ 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; @@ -43,16 +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.junit.Assert; -import org.junit.Test; +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 = @@ -61,8 +162,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 +191,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 +220,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/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()); + } } 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/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/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/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/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/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.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/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7d991ee603c9..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 @@ -18,10 +18,13 @@ */ 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; -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; @@ -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; @@ -54,10 +61,14 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; 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(); @@ -81,12 +92,17 @@ 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()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) .flinkConfig(config); } @@ -103,6 +119,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 + RecordCounterToWait.waitForCondition(); + CompletableFuture savepoint = + miniClusterResource + .getClusterClient() + .stopWithSavepoint( + jobId, + false, + TEMPORARY_FOLDER.newFolder().toPath().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 { testBoundedIcebergSource(FailoverType.TM); @@ -125,35 +190,17 @@ 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, 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)); @@ -222,6 +269,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 = + 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(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + } + // ------------------------------------------------------------------------ // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ @@ -263,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 f7dc931c506c..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 @@ -88,16 +97,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/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/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; 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 = 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); } } } diff --git a/flink/v1.16/build.gradle b/flink/v1.19/build.gradle similarity index 91% rename from flink/v1.16/build.gradle rename to flink/v1.19/build.gradle index 335a471e2455..a200a33890cc 100644 --- a/flink/v1.16/build.gradle +++ b/flink/v1.19/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.16' +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.flink116.avro + compileOnly libs.flink119.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.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,15 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } - testImplementation libs.flink116.connector.test.utils - testImplementation libs.flink116.core - testImplementation libs.flink116.runtime - testImplementation (libs.flink116.test.utilsjunit) { + 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.flink116.test.utils) { + testImplementation(libs.flink119.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -166,7 +168,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink116.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 +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.flink116.test.utils) { + integrationImplementation(libs.flink119.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.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.16/flink-runtime/LICENSE b/flink/v1.19/flink-runtime/LICENSE similarity index 100% rename from flink/v1.16/flink-runtime/LICENSE rename to flink/v1.19/flink-runtime/LICENSE diff --git a/flink/v1.16/flink-runtime/NOTICE b/flink/v1.19/flink-runtime/NOTICE similarity index 100% rename from flink/v1.16/flink-runtime/NOTICE rename to flink/v1.19/flink-runtime/NOTICE diff --git a/flink/v1.16/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 81% rename from flink/v1.16/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 index 65cfba1ec876..0d9bbf9d3601 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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 88% rename from flink/v1.16/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 index 825816fdf416..86295d78cc13 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -38,6 +38,8 @@ 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; @@ -60,8 +62,6 @@ 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; @@ -69,6 +69,7 @@ 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; @@ -91,7 +92,6 @@ * independent of the partition of Flink. */ public class FlinkCatalog extends AbstractCatalog { - private final CatalogLoader catalogLoader; private final Catalog icebergCatalog; private final Namespace baseNamespace; @@ -391,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()) { @@ -439,14 +438,35 @@ private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTab if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) && equalsPrimary)) { - throw new UnsupportedOperationException("Altering schema is not supported yet."); + 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 { @@ -464,12 +484,6 @@ public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean } 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(); @@ -507,7 +521,66 @@ public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean } }); - commitChanges(icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); + 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) { @@ -552,52 +625,6 @@ private static List toPartitionKeys(PartitionSpec spec, Schema icebergSc 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()); diff --git a/flink/v1.16/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 99% rename from flink/v1.16/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 index 1453753849ec..fe4008a13ce5 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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 92% rename from flink/v1.16/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 index 8e1f420b722d..b7f1be4b93fb 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v1.19/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 68% rename from flink/v1.16/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 index 25725639c330..4790dc85bf28 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.19/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; + } } /** @@ -134,6 +175,16 @@ 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}. * diff --git a/flink/v1.16/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.16/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.16/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 99% rename from flink/v1.16/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 index 6f8bfef2ef44..408065f06057 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java @@ -49,6 +49,10 @@ 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 diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 81% rename from flink/v1.16/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 index ab7b1174c9f3..ad4310a6d195 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.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.16/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.16/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.16/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 97% rename from flink/v1.16/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 index 40e0b5f2a34e..e8a46c5becd7 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 82% rename from flink/v1.16/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 index 9de0d6aaa551..5b28c4acb1c5 100644 --- a/flink/v1.16/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 @@ -21,8 +21,8 @@ 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; +import org.apache.iceberg.flink.util.ElapsedTimeGauge; class IcebergFilesCommitterMetrics { private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); @@ -70,27 +70,4 @@ void updateCommitSummary(CommitSummary stats) { 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.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java similarity index 100% rename from flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.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 new file mode 100644 index 000000000000..8a91411c1d21 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.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.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. + */ +class AggregatedStatistics implements Serializable { + private final long checkpointId; + private final StatisticsType type; + private final Map keyFrequency; + private final SortKey[] rangeBounds; + + 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; + } + + 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); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .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 new file mode 100644 index 000000000000..52d8a2f16f99 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.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.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; + +/** + * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress + * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific + * 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 Comparator comparator; + private final NavigableMap aggregationsPerCheckpoint; + + private AggregatedStatistics completedStatistics; + + AggregatedStatisticsTracker( + String operatorName, + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable AggregatedStatistics 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.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.aggregationsPerCheckpoint = Maps.newTreeMap(); + } + + 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 (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, + 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 { + 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(); + } + + 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.16/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 74% rename from flink/v1.16/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 index 9d7cf179ab1c..76c59cd5f4b8 100644 --- a/flink/v1.16/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.16/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 78% rename from flink/v1.16/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 index c8ac79c61bf6..6e9e4f6fa6c2 100644 --- a/flink/v1.16/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.16/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 69% rename from flink/v1.16/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 index 47dbfc3cfbe1..ffb428283785 100644 --- a/flink/v1.16/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.16/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 51% rename from flink/v1.16/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 index 5157a37cf2cd..2910471762cd 100644 --- a/flink/v1.16/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.16/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 58% rename from flink/v1.16/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 index 0ffffd9cf49f..22dd2388cc3e 100644 --- a/flink/v1.16/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.16/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 97% rename from flink/v1.16/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 index fb1a8f03a65c..4f52915a925e 100644 --- a/flink/v1.16/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.16/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 95% rename from flink/v1.16/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 index d03409f2a430..9c0e65a1fc49 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 97% rename from flink/v1.16/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 index fa1656c55278..b1431a32dd20 100644 --- a/flink/v1.16/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.16/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.16/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.16/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 98% rename from flink/v1.16/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 index 0655cf87a996..48201ea09359 100644 --- a/flink/v1.16/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); } } @@ -500,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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 97% rename from flink/v1.16/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 index cf57a126ae59..ab79a3173933 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 95% rename from flink/v1.16/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 index 37a0f1a6055f..e7447d08c985 100644 --- a/flink/v1.16/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 @@ -103,6 +103,13 @@ 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); diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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 95% rename from flink/v1.16/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 index ca60612f0ec9..dae7c8cca70c 100644 --- a/flink/v1.16/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 @@ -115,4 +115,10 @@ default void onCompletedSplits(Collection completedSplitIds) {} * 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.16/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.16/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.16/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.16/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.16/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 94% rename from flink/v1.16/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 index 3aca390755ed..6c9a855bc149 100644 --- a/flink/v1.16/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 @@ -36,10 +36,6 @@ 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); @@ -55,6 +51,12 @@ abstract class AbstractIcebergEnumerator 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 diff --git a/flink/v1.16/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.16/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.16/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 94% rename from flink/v1.16/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 index 55451b105885..c7021b9c6847 100644 --- a/flink/v1.16/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 @@ -21,6 +21,7 @@ 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; @@ -28,6 +29,7 @@ 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; @@ -58,6 +60,8 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { /** 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, @@ -72,6 +76,10 @@ public ContinuousIcebergEnumerator( 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()); @@ -140,6 +148,7 @@ private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwab 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. diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 90% rename from flink/v1.16/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 index 7ee75d2ade01..9c20494fdbcd 100644 --- a/flink/v1.16/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 @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Queue; @@ -132,6 +133,16 @@ public void close() throws Exception { } } + @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); } diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 82% rename from flink/v1.16/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 index e4bfbf1452e2..44e37afcfc60 100644 --- a/flink/v1.16/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.16/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 92% rename from flink/v1.16/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 index 8c089819e731..d4b0f9e1977d 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.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.16/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.16/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.16/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.16/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.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.16/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 91% rename from flink/v1.16/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 index 2c5c587f4ebf..f02af894e82b 100644 --- a/flink/v1.16/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 @@ -20,6 +20,7 @@ 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; @@ -39,4 +40,8 @@ public static TypeInformation toTypeInfo(RowType rowType) { public static boolean isPhysicalColumn(TableColumn column) { return column.isPhysical(); } + + public static boolean isPhysicalColumn(Column column) { + return column.isPhysical(); + } } diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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 98% rename from flink/v1.16/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 index 47319ec9bc60..4184526a6a1a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java +++ b/flink/v1.19/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 77% rename from flink/v1.16/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 index 45af9241b743..399d7aaff64c 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.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.util.MiniClusterWithClientResource; @@ -50,4 +51,18 @@ public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() .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.16/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.16/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.16/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.16/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.16/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 98% rename from flink/v1.16/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 index f48764f772b4..ce6caca12158 100644 --- a/flink/v1.16/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.16/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 84% rename from flink/v1.16/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 index 3986f1a796a5..e0b429b31b64 100644 --- a/flink/v1.16/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.16/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 86% rename from flink/v1.16/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 index 384ac5c52d00..e8f65921c19a 100644 --- a/flink/v1.16/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.16/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.16/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.16/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 86% rename from flink/v1.16/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 index 8e9066e391c9..1997ef6998a2 100644 --- a/flink/v1.16/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,16 +88,16 @@ 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); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); 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.16/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 94% rename from flink/v1.16/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 index e9372adda4c1..8992cbd75187 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ b/flink/v1.19/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.16/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.16/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.16/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 85% rename from flink/v1.16/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 index 2fbd7627efab..70c8043f8fbb 100644 --- a/flink/v1.16/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.16/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 97% rename from flink/v1.16/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 index f46d50a5f0ab..38379967044a 100644 --- a/flink/v1.16/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.16/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 88% rename from flink/v1.16/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 index c3f8bf92e47a..4c9e95b8fa82 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.19/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 @@ -85,7 +87,8 @@ public void testCreateCatalogCustomWithHiveCatalogTypeSet() { props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - Assertions.assertThatThrownBy( + + assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith( @@ -96,7 +99,8 @@ public void testCreateCatalogCustomWithHiveCatalogTypeSet() { 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.16/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 51% rename from flink/v1.16/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 index 3442ae8f9b16..f3af2c3cbe98 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -30,6 +30,7 @@ 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; @@ -73,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(); } @@ -225,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')"); @@ -278,7 +292,7 @@ public void testLoadTransformPartitionTable() throws TableNotExistException { } @TestTemplate - public void testAlterTable() throws TableNotExistException { + public void testAlterTableProperties() throws TableNotExistException { sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); Map properties = Maps.newHashMap(); properties.put("oldK", "oldV"); @@ -294,39 +308,284 @@ public void testAlterTable() throws TableNotExistException { assertThat(table("tl").properties()).containsAllEntriesOf(properties); // remove property - CatalogTable catalogTable = catalogTable("tl"); + sql("ALTER TABLE tl RESET('oldK')"); 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"); + 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."); + } - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); + @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."); - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); + // 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."); + } - // 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 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 diff --git a/flink/v1.16/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 96% rename from flink/v1.16/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 index 8fe406747169..a6feb2607774 100644 --- a/flink/v1.16/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(); } @@ -91,7 +91,8 @@ public void testListPartitionsWithUnpartitionedTable() { FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) .isInstanceOf(TableNotPartitionedException.class) - .hasMessage("Table " + objectPath + " in catalog " + catalogName + " is not partitioned."); + .hasMessageStartingWith("Table db.test_table in catalog") + .hasMessageEndingWith("is not partitioned."); } @TestTemplate diff --git a/flink/v1.16/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.16/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.16/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 85% rename from flink/v1.16/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 index c89ea4f53054..838b0ea0e1a9 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java +++ b/flink/v1.19/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.16/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 79% rename from flink/v1.16/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 index 47ee2afceb02..91343ab1ee72 100644 --- a/flink/v1.16/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,18 +83,19 @@ 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"); + dropDatabase("test_db", false); dropCatalog("test_catalog", false); } } diff --git a/flink/v1.16/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 93% rename from flink/v1.16/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 index 16b220ba6715..eab60d886ada 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ b/flink/v1.19/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 @@ -407,9 +407,10 @@ public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { Types.StructType.of( 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"); + .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.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 99% rename from flink/v1.16/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 index b7fce104f490..3f66174049a4 100644 --- a/flink/v1.16/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.16/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 99% rename from flink/v1.16/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 index 5674c83e40b8..baf13017ff99 100644 --- a/flink/v1.16/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.16/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.16/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.16/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 80% rename from flink/v1.16/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 index cb409b784383..c978ec6f8bd4 100644 --- a/flink/v1.16/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,54 +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()); - - 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. + 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(); @@ -292,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", @@ -340,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.16/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 93% rename from flink/v1.16/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 index 6bd94e9ca61c..8f1f129e183b 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v1.19/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.16/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 85% rename from flink/v1.16/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 index c78fa51215dd..caefbb5a5429 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.19/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.16/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.16/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.16/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 91% rename from flink/v1.16/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 index 27124d93fef4..7f0e7acaa822 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java +++ b/flink/v1.19/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.16/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 99% rename from flink/v1.16/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 index 4220775f41fa..5d8cce07330a 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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 91% rename from flink/v1.16/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 index e8aab824ea2d..a1039d27d888 100644 --- a/flink/v1.16/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 @@ -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.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 87% rename from flink/v1.16/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 index fdffc0e01c20..72f2ce4f4bce 100644 --- a/flink/v1.16/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 @@ -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.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.16/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 88% rename from flink/v1.16/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 index 7b868eafc311..b1e6f5aa00ff 100644 --- a/flink/v1.16/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 @@ -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.16/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.16/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.16/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 64% rename from flink/v1.16/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 index df2e6ae21c7e..7dd4e8759c0e 100644 --- a/flink/v1.16/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 @@ -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.16/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 98% rename from flink/v1.16/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 index e0340e0743b0..eccab20e04fc 100644 --- a/flink/v1.16/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 @@ -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.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); + } +} diff --git a/flink/v1.16/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 94% rename from flink/v1.16/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 index 6a493692c20d..44eb907a17aa 100644 --- a/flink/v1.16/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 @@ -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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 99% rename from flink/v1.16/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 index 214c3de1063e..14f12422da96 100644 --- a/flink/v1.16/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 @@ -138,7 +138,7 @@ public void testCompressionParquet() throws Exception { resultProperties.get(TableProperties.PARQUET_COMPRESSION)); Assert.assertEquals( TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); } else { Assert.assertEquals( initProperties.get(TableProperties.PARQUET_COMPRESSION), diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 87% rename from flink/v1.16/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 index b5c3bcf41734..5e81c279b69b 100644 --- a/flink/v1.16/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()); + } } diff --git a/flink/v1.16/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 99% rename from flink/v1.16/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 index 507df9e35215..0b403756cefe 100644 --- a/flink/v1.16/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 @@ -244,6 +244,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { .hasMessageStartingWith( "In 'hash' distribution mode with equality fields set, partition field") .hasMessageContaining("should be included in equality fields:"); + } else { testChangeLogs( ImmutableList.of("id"), @@ -287,7 +288,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { expectedRecords, branch)) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("should be included in equality fields"); + .hasMessageContaining("should be included in equality fields:"); } } diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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 87% rename from flink/v1.16/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 index f79a3e634071..939ed2be7dbc 100644 --- a/flink/v1.16/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 @@ -18,9 +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; @@ -32,6 +35,14 @@ 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, diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.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/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 new file mode 100644 index 000000000000..52485a3c67d9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, 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.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")); + 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")); + } + + 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")); + } + } + + @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")); + } + } + + @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 + 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.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 = + 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.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 new file mode 100644 index 000000000000..fe1d07c3286a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.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.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.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.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 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(Fixtures.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); + + // 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, + 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, 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 new file mode 100644 index 000000000000..966c8474b42b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.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.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.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.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); + 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"), + 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 + 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); + + 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 + 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(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + aggregatedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(aggregatedStatistics.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)); + } else { + assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + } + } + } + + 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 new file mode 100644 index 000000000000..6c864791a1bc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.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.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.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.EnumSource; + +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) throws Exception { + MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + 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; + } + + @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))); + + 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); + 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))); + } + } + + @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 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); + + 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(type); + try (OneInputStreamOperatorTestHarness testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + testHarness2.setup(); + testHarness2.initializeState(snapshot); + + 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); + } + } + } + + @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); + + // 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 { + 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.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 new file mode 100644 index 000000000000..0ab7d7c3b2ba --- /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 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 { + @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); + 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.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java similarity index 96% rename from flink/v1.16/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 index 92eb71acc834..dc3bad83947a 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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 61% rename from flink/v1.16/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 index 291302aef486..54cceae6e55b 100644 --- a/flink/v1.16/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/flink/v1.16/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.16/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.16/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.16/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.16/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 83% rename from flink/v1.16/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 index 7aa2b8034bc5..d3748e008b8e 100644 --- a/flink/v1.16/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.16/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 64% rename from flink/v1.16/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 index 3a8071523b7c..ebd220b00dba 100644 --- a/flink/v1.16/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.16/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.16/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.16/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 77% rename from flink/v1.16/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 index 7b5f9328694c..bde751e1f87f 100644 --- a/flink/v1.16/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.16/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 89% rename from flink/v1.16/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 index ed3f54bec642..366a3c2b5087 100644 --- a/flink/v1.16/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.16/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 99% rename from flink/v1.16/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 index 226da5813ad8..1b4fc863631f 100644 --- a/flink/v1.16/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 @@ -35,6 +35,7 @@ import org.apache.iceberg.util.StructLikeSet; public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { + @Override protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) throws IOException { diff --git a/flink/v1.16/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.16/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.16/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 99% rename from flink/v1.16/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 index f58cc87c6a29..8352924d042a 100644 --- a/flink/v1.16/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.16/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 99% rename from flink/v1.16/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 index 2f0af1c3ba0c..0b5a8011ad3f 100644 --- a/flink/v1.16/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 @@ -41,6 +41,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { + protected static String databaseName = "default"; protected static HiveConf hiveConf = null; diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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 97% rename from flink/v1.16/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 index 86c7e8991db8..ff7892071231 100644 --- a/flink/v1.16/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 @@ -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.16/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 83% rename from flink/v1.16/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 index 1814ff8f8542..bc7194e38088 100644 --- a/flink/v1.16/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.16/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.16/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.16/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 53% rename from flink/v1.16/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 index b56e804d1400..facbdcaaa533 100644 --- a/flink/v1.16/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,26 +85,25 @@ 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); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); } @Test public void testLimitPushDown() { - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) - .as("Invalid limit number: -1 ") + + assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) .isInstanceOf(SqlParserException.class) - .hasMessageContaining("SQL parse failed. Encountered \"-\""); + .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,11 +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)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + 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); } @Test @@ -429,22 +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.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + 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 @@ -453,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 @@ -469,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 @@ -483,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 @@ -498,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 @@ -517,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 @@ -532,14 +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); + 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); } @Test @@ -547,46 +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 1 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()); - - 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()); + 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 @@ -596,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.16/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 86% rename from flink/v1.16/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 index 3c0c38e1115d..0bf09b1643be 100644 --- a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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 90% rename from flink/v1.16/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 index bfd7fa5758e3..5765b73a1f63 100644 --- a/flink/v1.16/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,10 +19,12 @@ 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; 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; @@ -30,7 +32,9 @@ 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; @@ -58,9 +62,11 @@ public class TestIcebergSourceContinuous { + public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); + @ClassRule public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); + MiniClusterResource.createWithClassloaderCheckDisabled(METRIC_REPORTER); @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -112,6 +118,8 @@ public void testTableScanThenIncremental() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -162,6 +170,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -211,6 +221,8 @@ public void testEarliestSnapshot() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -263,6 +275,8 @@ public void testLatestSnapshot() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -313,6 +327,8 @@ public void testSpecificSnapshotId() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -367,6 +383,8 @@ public void testSpecificSnapshotTimestamp() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -456,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(); @@ -505,4 +537,22 @@ public static List getRunningJobs(ClusterClient client) throws Excepti .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.16/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 75% rename from flink/v1.16/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 index 7d991ee603c9..a378a0b93be3 100644 --- a/flink/v1.16/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,10 +18,13 @@ */ 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; -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; @@ -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; @@ -54,10 +61,14 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; 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(); @@ -81,12 +92,17 @@ 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()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) .flinkConfig(config); } @@ -103,6 +119,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 + RecordCounterToWait.waitForCondition(); + CompletableFuture savepoint = + miniClusterResource + .getClusterClient() + .stopWithSavepoint( + jobId, + false, + TEMPORARY_FOLDER.newFolder().toPath().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 { testBoundedIcebergSource(FailoverType.TM); @@ -125,35 +190,17 @@ 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, 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)); @@ -222,6 +269,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 = + 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(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + } + // ------------------------------------------------------------------------ // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ @@ -263,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.16/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 85% rename from flink/v1.16/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 index f7dc931c506c..cde39ec98d61 100644 --- a/flink/v1.16/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 @@ -88,16 +97,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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 99% rename from flink/v1.16/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 index 40dfda723749..9cf953342a18 100644 --- a/flink/v1.16/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.16/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.16/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.16/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 93% rename from flink/v1.16/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 index 485035787d6d..6ef40693827e 100644 --- a/flink/v1.16/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 @@ -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.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.16/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.16/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.16/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.16/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.16/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 99% rename from flink/v1.16/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 index 9e043bbbbbd2..420bcd52bfda 100644 --- a/flink/v1.16/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; @@ -94,7 +96,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.16/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 99% rename from flink/v1.16/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 index f7b13598bc2a..9c4f476b02b4 100644 --- a/flink/v1.16/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 @@ -248,6 +248,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .monitorInterval(Duration.ofMillis(100)) .maxPlanningSnapshotCount(0) .build(); + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 99% rename from flink/v1.16/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 index bb747caae589..1bb2b1c6bf85 100644 --- a/flink/v1.16/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 @@ -337,6 +337,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { 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"); @@ -362,6 +363,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { 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); @@ -423,6 +425,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find a snapshot after: 1"); @@ -443,9 +446,10 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); + .hasMessageStartingWith("Cannot find a snapshot after:"); } @Test diff --git a/flink/v1.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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.16/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 89% rename from flink/v1.16/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 index cd778309f90d..c72d622f86ba 100644 --- a/flink/v1.16/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 = diff --git a/flink/v1.16/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 85% rename from flink/v1.16/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 index 9d99193fb1be..4ba4f9d983dc 100644 --- a/flink/v1.16/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 @@ -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.16.3", FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo("1.19.0"); } @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); } } } diff --git a/flink/v1.16/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.16/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 diff --git a/format/spec.md b/format/spec.md index ab6f3494830c..6e23b2c5834b 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: @@ -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 @@ -314,7 +315,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 +352,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 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 41b87a22faf4..a5875a75ecf6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -23,26 +23,27 @@ 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" +assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.25.21" -azuresdk-bom = "1.2.22" +awssdk-bom = "2.25.64" +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.1.0" +delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.26.1" +errorprone-annotations = "2.28.0" 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" +guava = "33.2.1-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" httpcomponents-httpclient5 = "5.3.1" @@ -65,25 +66,25 @@ kryo-shaded = "4.0.3" 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" +nessie = "0.83.2" +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" 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" -snowflake-jdbc = "3.14.5" +snowflake-jdbc = "3.16.0" 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.33" -sqlite-jdbc = "3.45.2.0" -testcontainers = "1.19.5" +spring-web = "5.3.36" +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 @@ -102,15 +103,10 @@ 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" } -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 +119,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" } @@ -142,8 +144,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" } @@ -175,11 +178,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 +188,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/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/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/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 75d59de75d4d..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(); @@ -235,13 +235,14 @@ 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(); try { persistTable( - tbl, updateHiveTable, hiveLockEnabled(metadata, conf) ? null : baseMetadataLocation); + tbl, updateHiveTable, hiveLockEnabled(base, conf) ? null : baseMetadataLocation); lock.ensureActive(); commitStatus = BaseMetastoreOperations.CommitStatus.SUCCESS; @@ -509,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/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); 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(); 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 { 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/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 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/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() { 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")); } } 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' } 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/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/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 * 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(); + } +} 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 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 7bd97b69885f..c5af1940c324 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): @@ -361,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 """ @@ -373,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` """ @@ -382,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 """ @@ -392,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` """ @@ -401,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` """ @@ -410,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` """ @@ -419,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` """ @@ -428,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` """ @@ -437,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` """ @@ -844,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 161d5e0fcff8..4bb73cd44120 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: @@ -1378,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: @@ -1986,8 +1994,8 @@ components: operation: type: string enum: ["append", "replace", "overwrite", "delete"] - additionalProperties: - type: string + additionalProperties: + type: string schema-id: type: integer @@ -2578,6 +2586,7 @@ components: - $ref: '#/components/schemas/SetCurrentViewVersionUpdate' TableRequirement: + type: object discriminator: propertyName: type mapping: @@ -2589,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: @@ -2609,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 @@ -2623,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 @@ -2643,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: @@ -2658,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: @@ -2673,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: @@ -2688,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: @@ -2703,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: @@ -2718,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 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/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/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(); 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 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}" 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/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/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 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 diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index f65b9d6c3941..0d2bc48464a6 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 @@ -310,77 +303,23 @@ 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. -### 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 +#### Update DOAP (ASF Project Description) -Once the common docs changes have been merged into `main`, the next step is to update the versioned docs. +- Create a PR to update the release version in [doap.rdf](https://github.com/apache/iceberg/blob/main/doap.rdf) file, in the `` section: -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 +```xml + + + x.y.z + yyyy-mm-dd + x.y.z + + ``` -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 -``` +### Documentation Release -#### 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 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) | diff --git a/site/docs/releases.md b/site/docs/releases.md index 09b7f439385b..6ba0b481fce9 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) @@ -36,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. @@ -69,6 +71,29 @@ 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. + +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 @@ -974,3 +999,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) + + diff --git a/site/docs/vendors.md b/site/docs/vendors.md index d549219e5c10..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/) @@ -71,3 +78,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. diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 45df8d93bd39..899851984cad 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.5.0' + icebergVersion: '1.5.2' nessieVersion: '0.77.1' social: - icon: fontawesome/regular/comments diff --git a/site/nav.yml b/site/nav.yml index 4a3548433636..3fe17a94953f 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -23,11 +23,14 @@ 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' - 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 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

    diff --git a/site/requirements.txt b/site/requirements.txt index e170b507fb44..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.15 +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 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..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 @@ -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; @@ -246,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( @@ -255,8 +257,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 +289,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..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,13 +18,16 @@ */ 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; 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 +337,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(); @@ -397,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-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/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/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.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.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/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.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/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.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/source/TestStructuredStreamingRead3.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index a2d0c9acaf48..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 @@ -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 @@ -560,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.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.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-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/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/SparkRowLevelOperationsTestBase.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index e4605a765823..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( @@ -292,8 +293,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 +325,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/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-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..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,7 +18,9 @@ */ 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; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_MODE; @@ -27,6 +29,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 +505,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(); @@ -565,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-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-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..49119d319c40 --- /dev/null +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.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.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(); + 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-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..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 @@ -24,6 +24,7 @@ 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; @@ -165,6 +166,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"); + Assertions.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.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/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/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.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() { 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.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.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/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"; 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; }); } 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/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, 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/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.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/source/TestStructuredStreamingRead3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 1c08744f5662..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 @@ -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 @@ -565,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.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.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-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/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-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..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( @@ -285,8 +286,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 +319,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/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-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..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,7 +18,9 @@ */ 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; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_MODE; @@ -501,6 +503,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(); @@ -564,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-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-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(); } 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..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 @@ -174,6 +174,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"); + Assertions.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/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/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/SparkConfParser.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java index e3b01b8375b6..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 @@ -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,18 @@ 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; + } else { + return new CaseInsensitiveStringMap(map); + } + } + class BooleanConfParser extends ConfParser { private Boolean defaultValue; private boolean negate = false; @@ -197,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; @@ -220,14 +265,15 @@ 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); + } + + String sparkOptionValue = options.get(toCamelCase(optionName)); + if (sparkOptionValue != null) { + return conversion.apply(sparkOptionValue); } } @@ -236,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) { @@ -247,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/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 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..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 @@ -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); } @@ -302,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() { 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()); } /** 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/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. */ 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/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/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; }); } 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/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, 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/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/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); + } } 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); } 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..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 @@ -79,6 +79,41 @@ 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 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); 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); 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); + } + } } 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)); + } +} 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/source/TestStructuredStreamingRead3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 22e7df0f4e17..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 @@ -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 @@ -646,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. 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 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)