diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 1491a35db194..570fb84d70a4 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -414,6 +414,9 @@ + + + diff --git a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml index f3d6d6de923a..80048f99465a 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml @@ -50,4 +50,12 @@ body: - Hive - Other validations: - required: false \ No newline at end of file + required: false + - type: checkboxes + attributes: + label: Willingness to contribute + description: The Apache Iceberg community encourages contributions. Would you or another member of your organization be willing to contribute this improvement/feature to the Apache Iceberg codebase? + options: + - label: I can contribute this improvement/feature independently + - label: I would be willing to contribute this improvement/feature with guidance from the Iceberg community + - label: I cannot contribute this improvement/feature at this time diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index ac5314e8afb8..cd16847cf95a 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -71,7 +71,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -100,7 +100,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index d2e249c279f7..370375783cc2 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -73,15 +73,8 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] - flink: ['1.17', '1.18', '1.19'] - exclude: - # Flink 1.17 does not support Java 17. - - jvm: 17 - flink: '1.17' - # Flink 1.17 does not support Java 21. - - jvm: 21 - flink: '1.17' + jvm: [11, 17, 21] + flink: ['1.18', '1.19', '1.20'] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index ee487807cff4..6ad9f58410d6 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -69,7 +69,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -98,7 +98,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index e1fd90116ef1..0d39ee8646ad 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -65,7 +65,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -94,7 +94,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 @@ -107,7 +107,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 411c3a5636c9..6acee54bde3e 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -37,7 +37,7 @@ jobs: - uses: actions/setup-java@v4 with: distribution: zulu - java-version: 8 + java-version: 11 - run: | ./gradlew printVersion ./gradlew -DallModules publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 7a47beeed7a4..1cc0425b73a3 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -71,7 +71,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] spark: ['3.3', '3.4', '3.5'] scala: ['2.12', '2.13'] exclude: diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 3018840b4513..50d98690c9b7 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1056,6 +1056,87 @@ acceptedBreaks: - code: "java.method.removed" old: "method org.apache.iceberg.DataFiles.Builder org.apache.iceberg.DataFiles.Builder::withEqualityFieldIds(java.util.List)" justification: "Deprecations for 1.6.0 release" + "1.6.0": + org.apache.iceberg:iceberg-common: + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynFields.StaticField org.apache.iceberg.common.DynFields.Builder::buildStaticChecked()\ + \ throws java.lang.NoSuchFieldException" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method java.lang.Class org.apache.iceberg.common.DynConstructors.Ctor::getConstructedClass()" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynConstructors.Builder org.apache.iceberg.common.DynConstructors.Builder::hiddenImpl(java.lang.Class[])" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynMethods.Builder org.apache.iceberg.common.DynMethods.Builder::ctorImpl(java.lang.Class,\ + \ java.lang.Class[])" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynMethods.Builder org.apache.iceberg.common.DynMethods.Builder::ctorImpl(java.lang.String,\ + \ java.lang.Class[])" + justification: "Removing deprecated code" + - code: "java.method.visibilityReduced" + old: "method R org.apache.iceberg.common.DynMethods.UnboundMethod::invokeChecked(java.lang.Object,\ + \ java.lang.Object[]) throws java.lang.Exception" + new: "method R org.apache.iceberg.common.DynMethods.UnboundMethod::invokeChecked(java.lang.Object,\ + \ java.lang.Object[]) throws java.lang.Exception" + justification: "Reduced visibility and scoped to package" + org.apache.iceberg:iceberg-core: + - code: "java.class.removed" + old: "enum org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus" + justification: "Removing deprecated code" + - code: "java.method.addedToInterface" + new: "method org.apache.iceberg.metrics.CounterResult org.apache.iceberg.metrics.CommitMetricsResult::totalDataManifestFiles()" + justification: "Added new parameters for manifest stats" + - code: "java.method.addedToInterface" + new: "method org.apache.iceberg.metrics.CounterResult org.apache.iceberg.metrics.CommitMetricsResult::totalDeleteManifestFiles()" + justification: "Added new parameters for manifest stats" + - code: "java.method.removed" + old: "method java.lang.String org.apache.iceberg.FileScanTaskParser::toJson(org.apache.iceberg.FileScanTask)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.FileScanTask org.apache.iceberg.FileScanTaskParser::fromJson(java.lang.String,\ + \ boolean)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.ContentCache.CacheEntry org.apache.iceberg.io.ContentCache::get(java.lang.String,\ + \ java.util.function.Function)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.ContentCache.CacheEntry org.apache.iceberg.io.ContentCache::getIfPresent(java.lang.String)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.InputFile org.apache.iceberg.io.ContentCache::tryCache(org.apache.iceberg.io.FileIO,\ + \ java.lang.String, long)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.BaseRewriteManifests" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, java.lang.String, java.lang.String, java.lang.String,\ + \ java.lang.String, java.lang.String)" + justification: "Removing deprecated code" + - code: "java.method.returnTypeChanged" + old: "method org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\ + \ org.apache.iceberg.TableMetadata)" + new: "method org.apache.iceberg.BaseMetastoreOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\ + \ org.apache.iceberg.TableMetadata)" + justification: "Removing deprecated code" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/README.md b/README.md index fe0d2b94c308..7d2056077804 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ Community discussions happen primarily on the [dev mailing list][dev-list] or on ### Building -Iceberg is built using Gradle with Java 8, 11, 17, or 21. +Iceberg is built using Gradle with Java 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java index 499e61495fc0..f7a4b72e4b97 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockLocalStore.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aliyun.oss.mock; +import static org.assertj.core.api.Assertions.assertThat; + import com.aliyun.oss.OSSErrorCode; import com.aliyun.oss.model.Bucket; import com.fasterxml.jackson.databind.ObjectMapper; @@ -137,7 +139,9 @@ ObjectMetadata putObject( Map userMetaData) throws IOException { File bucketDir = new File(root, bucketName); - assert bucketDir.exists() || bucketDir.mkdirs(); + assertThat(bucketDir) + .satisfiesAnyOf( + bucket -> assertThat(bucket).exists(), bucket -> assertThat(bucket.mkdirs()).isTrue()); File dataFile = new File(bucketDir, fileName + DATA_FILE); File metaFile = new File(bucketDir, fileName + META_FILE); @@ -170,17 +174,21 @@ ObjectMetadata putObject( void deleteObject(String bucketName, String filename) { File bucketDir = new File(root, bucketName); - assert bucketDir.exists(); + assertThat(bucketDir).exists(); File dataFile = new File(bucketDir, filename + DATA_FILE); File metaFile = new File(bucketDir, filename + META_FILE); - assert !dataFile.exists() || dataFile.delete(); - assert !metaFile.exists() || metaFile.delete(); + assertThat(dataFile) + .satisfiesAnyOf( + file -> assertThat(file).doesNotExist(), file -> assertThat(file.delete()).isTrue()); + assertThat(metaFile) + .satisfiesAnyOf( + file -> assertThat(file).doesNotExist(), file -> assertThat(file.delete()).isTrue()); } ObjectMetadata getObjectMetadata(String bucketName, String filename) throws IOException { File bucketDir = new File(root, bucketName); - assert bucketDir.exists(); + assertThat(bucketDir).exists(); File dataFile = new File(bucketDir, filename + DATA_FILE); if (!dataFile.exists()) { diff --git a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java index f48d590af1ce..eeb596d42d5c 100644 --- a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java @@ -122,4 +122,15 @@ public interface UpdatePartitionSpec extends PendingUpdate { * change conflicts with other additions, removals, or renames. */ UpdatePartitionSpec renameField(String name, String newName); + + /** + * Sets that the new partition spec will be NOT set as the default partition spec for the table, + * the default behavior is to do so. + * + * @return this for method chaining + */ + default UpdatePartitionSpec addNonDefaultSpec() { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement addNonDefaultSpec()"); + }; } diff --git a/api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.java b/api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.java new file mode 100644 index 000000000000..c402d2e68e7d --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.io; + +/** + * This interface is intended as an extension for FileIO implementations to provide additional + * best-effort recovery operations that can be useful for repairing corrupted tables where there are + * reachable files missing from disk. (e.g. a live manifest points to data file entry which no + * longer exists on disk) + */ +public interface SupportsRecoveryOperations { + + /** + * Perform a best-effort recovery of a file at a given path + * + * @param path Absolute path of file to attempt recovery for + * @return true if recovery was successful, false otherwise + */ + boolean recoverFile(String path); +} diff --git a/api/src/main/java/org/apache/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java index d09d9f5395ce..a803afac104f 100644 --- a/api/src/main/java/org/apache/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -173,6 +173,10 @@ public static Comparator charSequences() { return CharSeqComparator.INSTANCE; } + public static Comparator filePath() { + return FilePathComparator.INSTANCE; + } + private static class NullsFirst implements Comparator { private static final NullsFirst INSTANCE = new NullsFirst<>(); @@ -351,4 +355,41 @@ public int compare(CharSequence s1, CharSequence s2) { return Integer.compare(s1.length(), s2.length()); } } + + private static class FilePathComparator implements Comparator { + private static final FilePathComparator INSTANCE = new FilePathComparator(); + + private FilePathComparator() {} + + @Override + public int compare(CharSequence s1, CharSequence s2) { + if (s1 == s2) { + return 0; + } + int count = s1.length(); + + int cmp = Integer.compare(count, s2.length()); + if (cmp != 0) { + return cmp; + } + + if (s1 instanceof String && s2 instanceof String) { + cmp = Integer.compare(s1.hashCode(), s2.hashCode()); + if (cmp != 0) { + return cmp; + } + } + // File paths inside a delete file normally have more identical chars at the beginning. For + // example, a typical + // path is like "s3:/bucket/db/table/data/partition/00000-0-[uuid]-00001.parquet". + // The uuid is where the difference starts. So it's faster to find the first diff backward. + for (int i = count - 1; i >= 0; i--) { + cmp = Character.compare(s1.charAt(i), s2.charAt(i)); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java index bbe062d5db48..7e0ca6ed10b2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; @@ -30,9 +31,10 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.Delete; import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.ListObjectVersionsRequest; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.ObjectVersion; +import software.amazon.awssdk.services.s3.paginators.ListObjectVersionsIterable; import software.amazon.awssdk.services.s3control.S3ControlClient; import software.amazon.awssdk.services.s3control.model.CreateAccessPointRequest; import software.amazon.awssdk.services.s3control.model.DeleteAccessPointRequest; @@ -94,28 +96,46 @@ public static String testAccountId() { } public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { - boolean hasContent = true; - while (hasContent) { - ListObjectsV2Response response = - s3.listObjectsV2( - ListObjectsV2Request.builder().bucket(bucketName).prefix(prefix).build()); - hasContent = response.hasContents(); - if (hasContent) { - s3.deleteObjects( - DeleteObjectsRequest.builder() - .bucket(bucketName) - .delete( - Delete.builder() - .objects( - response.contents().stream() - .map(obj -> ObjectIdentifier.builder().key(obj.key()).build()) - .collect(Collectors.toList())) - .build()) - .build()); - } + ListObjectVersionsIterable response = + s3.listObjectVersionsPaginator( + ListObjectVersionsRequest.builder().bucket(bucketName).prefix(prefix).build()); + List versionsToDelete = Lists.newArrayList(); + int batchDeletionSize = 1000; + response.versions().stream() + .forEach( + version -> { + versionsToDelete.add(version); + if (versionsToDelete.size() == batchDeletionSize) { + deleteObjectVersions(s3, bucketName, versionsToDelete); + versionsToDelete.clear(); + } + }); + + if (!versionsToDelete.isEmpty()) { + deleteObjectVersions(s3, bucketName, versionsToDelete); } } + private static void deleteObjectVersions( + S3Client s3, String bucket, List objectVersions) { + s3.deleteObjects( + DeleteObjectsRequest.builder() + .bucket(bucket) + .delete( + Delete.builder() + .objects( + objectVersions.stream() + .map( + obj -> + ObjectIdentifier.builder() + .key(obj.key()) + .versionId(obj.versionId()) + .build()) + .collect(Collectors.toList())) + .build()) + .build()); + } + public static void cleanGlueCatalog(GlueClient glue, List namespaces) { for (String namespace : namespaces) { try { diff --git a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java index 18abb82ce74a..cacf04891896 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java @@ -53,14 +53,17 @@ import software.amazon.awssdk.services.kms.model.ListAliasesResponse; import software.amazon.awssdk.services.kms.model.ScheduleKeyDeletionRequest; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.BucketVersioningStatus; import software.amazon.awssdk.services.s3.model.GetObjectAclRequest; import software.amazon.awssdk.services.s3.model.GetObjectAclResponse; import software.amazon.awssdk.services.s3.model.GetObjectRequest; import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; import software.amazon.awssdk.services.s3.model.Permission; +import software.amazon.awssdk.services.s3.model.PutBucketVersioningRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.ServerSideEncryption; +import software.amazon.awssdk.services.s3.model.VersioningConfiguration; import software.amazon.awssdk.services.s3control.S3ControlClient; import software.amazon.awssdk.utils.ImmutableMap; import software.amazon.awssdk.utils.IoUtils; @@ -106,6 +109,12 @@ public static void beforeClass() { AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); AwsIntegTestUtil.createAccessPoint( crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) + .build()); } @AfterAll @@ -445,6 +454,35 @@ public void testPrefixDelete() { }); } + @Test + public void testFileRecoveryHappyPath() throws Exception { + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); + String filePath = String.format("s3://%s/%s/%s", bucketName, prefix, "someFile.parquet"); + write(s3FileIO, filePath); + s3FileIO.deleteFile(filePath); + assertThat(s3FileIO.newInputFile(filePath).exists()).isFalse(); + + assertThat(s3FileIO.recoverFile(filePath)).isTrue(); + assertThat(s3FileIO.newInputFile(filePath).exists()).isTrue(); + } + + @Test + public void testFileRecoveryFailsToRecover() throws Exception { + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.SUSPENDED).build()) + .build()); + String filePath = String.format("s3://%s/%s/%s", bucketName, prefix, "unversionedFile.parquet"); + write(s3FileIO, filePath); + s3FileIO.deleteFile(filePath); + assertThat(s3FileIO.newInputFile(filePath).exists()).isFalse(); + + assertThat(s3FileIO.recoverFile(filePath)).isFalse(); + } + private S3FileIOProperties getDeletionTestProperties() { S3FileIOProperties properties = new S3FileIOProperties(); properties.setDeleteBatchSize(deletionBatchSize); diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java index dd13e13f01a6..f7d2da5eb907 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java @@ -20,8 +20,10 @@ import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -37,6 +39,7 @@ import org.apache.iceberg.io.FileInfo; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsRecoveryOperations; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -52,6 +55,7 @@ import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.Delete; import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; @@ -61,10 +65,12 @@ import software.amazon.awssdk.services.s3.model.GetObjectTaggingResponse; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.model.ObjectVersion; import software.amazon.awssdk.services.s3.model.PutObjectTaggingRequest; import software.amazon.awssdk.services.s3.model.S3Exception; import software.amazon.awssdk.services.s3.model.Tag; import software.amazon.awssdk.services.s3.model.Tagging; +import software.amazon.awssdk.services.s3.paginators.ListObjectVersionsIterable; /** * FileIO implementation backed by S3. @@ -73,7 +79,7 @@ * schemes s3a, s3n, https are also treated as s3 file paths. Using this FileIO with other schemes * will result in {@link org.apache.iceberg.exceptions.ValidationException}. */ -public class S3FileIO implements CredentialSupplier, DelegateFileIO { +public class S3FileIO implements CredentialSupplier, DelegateFileIO, SupportsRecoveryOperations { private static final Logger LOG = LoggerFactory.getLogger(S3FileIO.class); private static final String DEFAULT_METRICS_IMPL = "org.apache.iceberg.hadoop.HadoopMetricsContext"; @@ -420,4 +426,46 @@ protected void finalize() throws Throwable { } } } + + @Override + public boolean recoverFile(String path) { + S3URI location = new S3URI(path, s3FileIOProperties.bucketToAccessPointMapping()); + ListObjectVersionsIterable response = + client() + .listObjectVersionsPaginator( + builder -> builder.bucket(location.bucket()).prefix(location.key())); + + // Recover to the last modified version, not isLatest, + // since isLatest is true for deletion markers. + Optional recoverVersion = + response.versions().stream().max(Comparator.comparing(ObjectVersion::lastModified)); + + return recoverVersion.map(version -> recoverObject(version, location.bucket())).orElse(false); + } + + private boolean recoverObject(ObjectVersion version, String bucket) { + if (version.isLatest()) { + return true; + } + + LOG.info("Attempting to recover object {}", version.key()); + try { + // Perform a copy instead of deleting the delete marker + // so that recovery does not rely on delete permissions + client() + .copyObject( + builder -> + builder + .sourceBucket(bucket) + .sourceKey(version.key()) + .sourceVersionId(version.versionId()) + .destinationBucket(bucket) + .destinationKey(version.key())); + } catch (SdkException e) { + LOG.warn("Failed to recover object {}", version.key(), e); + return false; + } + + return true; + } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java index 135eb76772cc..38489e367434 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java @@ -81,6 +81,7 @@ import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; +import software.amazon.awssdk.services.s3.model.BucketAlreadyOwnedByYouException; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; @@ -447,7 +448,7 @@ private void createRandomObjects(String prefix, int count) { private void createBucket(String bucketName) { try { s3.get().createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); - } catch (BucketAlreadyExistsException e) { + } catch (BucketAlreadyExistsException | BucketAlreadyOwnedByYouException e) { // do nothing } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java index 95f4d09a7e0b..ed71e259a26c 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java @@ -35,6 +35,7 @@ import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; +import software.amazon.awssdk.services.s3.model.BucketAlreadyOwnedByYouException; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; @@ -195,7 +196,7 @@ private void writeS3Data(S3URI uri, byte[] data) throws IOException { private void createBucket(String bucketName) { try { s3.createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); - } catch (BucketAlreadyExistsException e) { + } catch (BucketAlreadyExistsException | BucketAlreadyOwnedByYouException e) { // don't do anything } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java index 6fbe59e47f7f..88488bf4c313 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3OutputStream.java @@ -62,6 +62,7 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; +import software.amazon.awssdk.services.s3.model.BucketAlreadyOwnedByYouException; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.GetObjectRequest; @@ -339,7 +340,7 @@ private S3URI randomURI() { private void createBucket(String bucketName) { try { s3.createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); - } catch (BucketAlreadyExistsException e) { + } catch (BucketAlreadyExistsException | BucketAlreadyOwnedByYouException e) { // do nothing } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java index 06c099e3be5e..ce7527af765c 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/S3SignerServlet.java @@ -148,7 +148,7 @@ private OAuthTokenResponse handleOAuth(Map requestMap) { .withToken("client-credentials-token:sub=" + requestMap.get("client_id")) .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") .withTokenType("Bearer") - .setExpirationInSeconds(100) + .setExpirationInSeconds(10000) .build()); case "urn:ietf:params:oauth:grant-type:token-exchange": @@ -163,7 +163,7 @@ private OAuthTokenResponse handleOAuth(Map requestMap) { .withToken(token) .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") .withTokenType("Bearer") - .setExpirationInSeconds(100) + .setExpirationInSeconds(10000) .build()); default: diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java index 5e20b71e438c..08f356ca7ab1 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java @@ -107,7 +107,13 @@ public static void afterClass() throws Exception { ScheduledThreadPoolExecutor executor = ((ScheduledThreadPoolExecutor) validatingSigner.icebergSigner.tokenRefreshExecutor()); - // token expiration is set to 100s so there should be exactly one token scheduled for refresh + // token expiration is set to 10000s by the S3SignerServlet so there should be exactly one token + // scheduled for refresh. Such a high token expiration value is explicitly selected to be much + // larger than TestS3RestSigner would need to execute all tests. + // The reason why this check is done here with a high token expiration is to make sure that + // there aren't other token refreshes being scheduled after every sign request and after + // TestS3RestSigner completes all tests, there should be only this single token in the queue + // that is scheduled for refresh assertThat(executor.getPoolSize()).isEqualTo(1); assertThat(executor.getQueue()) .as("should only have a single token scheduled for refresh") diff --git a/baseline.gradle b/baseline.gradle index 62ade9a632f4..42be6d8a24c6 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -42,9 +42,14 @@ subprojects { apply plugin: 'com.palantir.baseline-error-prone' } apply plugin: 'com.palantir.baseline-class-uniqueness' - apply plugin: 'com.palantir.baseline-reproducibility' + // What 'com.palantir.baseline-reproducibility' used to do, except the check for the + // `sourceCompatibility` Java compile option, which conflicts with the `release` compile option. + tasks.withType(AbstractArchiveTask.class).configureEach(t -> { + t.setPreserveFileTimestamps(false); + t.setReproducibleFileOrder(true); + t.setDuplicatesStrategy(DuplicatesStrategy.WARN); + }); apply plugin: 'com.palantir.baseline-exact-dependencies' - apply plugin: 'com.palantir.baseline-release-compatibility' // We need to update Google Java Format to 1.17.0+ to run spotless on JDK 8, but that requires dropping support for JDK 8. if (JavaVersion.current() == JavaVersion.VERSION_21) { task spotlessApply { @@ -86,11 +91,16 @@ subprojects { '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', + // TODO (https://github.com/apache/iceberg/issues/10853) this is a recently added check. Figure out whether we adjust the code or suppress for good + '-Xep:DangerousJavaDeserialization:WARN', '-Xep:DangerousThreadPoolExecutorUsage:OFF', // subclasses are not equal '-Xep:EqualsGetClass:OFF', // specific to Palantir '-Xep:FinalClass:OFF', + '-Xep:FormatStringAnnotation:ERROR', + // TODO (https://github.com/apache/iceberg/issues/10855) this is a recently added check. Figure out whether we adjust the code or suppress for good + '-Xep:ImmutablesReferenceEquality:WARN', '-Xep:IntLongMath:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', @@ -103,6 +113,8 @@ subprojects { '-Xep:MissingSummary:ERROR', // Enforce hashCode over hash '-Xep:ObjectsHashCodeUnnecessaryVarargs:ERROR', + // Triggers false-positives whenever relocated @VisibleForTesting is used + '-Xep:PreferCommonAnnotations:OFF', // specific to Palantir '-Xep:PreferSafeLoggableExceptions:OFF', '-Xep:PreferSafeLogger:OFF', @@ -120,6 +132,8 @@ subprojects { '-Xep:StringSplitter:ERROR', '-Xep:TypeParameterShadowing:OFF', '-Xep:TypeParameterUnusedInFormals:OFF', + // Palantir's UnnecessarilyQualified may throw during analysis + '-Xep:UnnecessarilyQualified:OFF', ) } } diff --git a/build.gradle b/build.gradle index 09a04e3ae977..7a11943cf8be 100644 --- a/build.gradle +++ b/build.gradle @@ -27,12 +27,7 @@ buildscript { } dependencies { classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' - classpath 'com.palantir.baseline:gradle-baseline-java:4.42.0' - // com.palantir.baseline:gradle-baseline-java:4.42.0 (the last version supporting Java 8) pulls - // in an old version of the errorprone, which doesn't work w/ Gradle 8, so bump errorpone as - // well. - classpath "net.ltgt.gradle:gradle-errorprone-plugin:3.1.0" - + classpath 'com.palantir.baseline:gradle-baseline-java:5.61.0' classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' @@ -56,10 +51,7 @@ try { project.logger.error(e.getMessage()) } -if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - project.ext.jdkVersion = '8' - project.ext.extraJvmArgs = [] -} else if (JavaVersion.current() == JavaVersion.VERSION_11) { +if (JavaVersion.current() == JavaVersion.VERSION_11) { project.ext.jdkVersion = '11' project.ext.extraJvmArgs = [] } else if (JavaVersion.current() == JavaVersion.VERSION_17 || JavaVersion.current() == JavaVersion.VERSION_21) { @@ -86,7 +78,7 @@ if (JavaVersion.current() == JavaVersion.VERSION_1_8) { "--add-opens", "java.base/sun.security.action=ALL-UNNAMED", "--add-opens", "java.base/sun.util.calendar=ALL-UNNAMED"] } else { - throw new GradleException("This build must be run with JDK 8 or 11 or 17 or 21 but was executed with JDK " + JavaVersion.current()) + throw new GradleException("This build must be run with JDK 11 or 17 or 21 but was executed with JDK " + JavaVersion.current()) } tasks.withType(AbstractArchiveTask).configureEach { @@ -194,21 +186,15 @@ subprojects { testArtifacts } - compileJava { - options.encoding = "UTF-8" - } - - compileTestJava { + tasks.withType(JavaCompile.class).configureEach { options.encoding = "UTF-8" + options.release = 11 } javadoc { options.encoding = 'UTF-8' } - sourceCompatibility = '1.8' - targetCompatibility = '1.8' - dependencies { implementation libs.slf4j.api @@ -251,6 +237,10 @@ subprojects { plugins.withType(ScalaPlugin.class) { tasks.withType(ScalaCompile.class) { scalaCompileOptions.keepAliveMode.set(KeepAliveMode.DAEMON) + // `options.release` doesn't seem to work for ScalaCompile :( + sourceCompatibility = "11" + targetCompatibility = "11" + scalaCompileOptions.additionalParameters.add("-release:11") } } } @@ -347,7 +337,7 @@ project(':iceberg-core') { annotationProcessor libs.immutables.value compileOnly libs.immutables.value - implementation(libs.avro.avro) { + api(libs.avro.avro) { exclude group: 'org.tukaani' // xz compression is not supported } @@ -364,6 +354,7 @@ project(':iceberg-core') { } testImplementation libs.jetty.servlet + testImplementation libs.jakarta.servlet testImplementation libs.jetty.server testImplementation libs.mockserver.netty testImplementation libs.mockserver.client.java @@ -887,23 +878,8 @@ project(':iceberg-pig') { } project(':iceberg-nessie') { - if (JavaVersion.current().isJava11Compatible()) { - test { - useJUnitPlatform() - } - compileTestJava { - sourceCompatibility = "11" - targetCompatibility = "11" - } - } else { - // Do not test Nessie against Java 8, because in-JVM testing requires Nessie server components, - // which require Java 11+. - test { - enabled = false - } - compileTestJava { - enabled = false - } + test { + useJUnitPlatform() } dependencies { @@ -922,21 +898,19 @@ project(':iceberg-nessie') { // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages compileOnly libs.microprofile.openapi.api - if (JavaVersion.current().isJava11Compatible()) { - testImplementation libs.nessie.jaxrs.testextension - testImplementation libs.nessie.versioned.storage.inmemory.tests - testImplementation libs.nessie.versioned.storage.testextension - // Need to "pull in" el-api explicitly :( - testImplementation libs.jakarta.el.api + testImplementation libs.nessie.jaxrs.testextension + testImplementation libs.nessie.versioned.storage.inmemory.tests + testImplementation libs.nessie.versioned.storage.testextension + // Need to "pull in" el-api explicitly :( + testImplementation libs.jakarta.el.api - testImplementation libs.avro.avro + testImplementation libs.avro.avro - testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages - testCompileOnly libs.microprofile.openapi.api - } + // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages + testCompileOnly libs.microprofile.openapi.api } } diff --git a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java index 7c777112871a..7ec8716c86a4 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java +++ b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java @@ -43,12 +43,6 @@ private Ctor(Constructor constructor, Class constructed) { this.constructed = constructed; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Class getConstructedClass() { - return constructed; - } - public C newInstanceChecked(Object... args) throws Exception { try { if (args.length > ctor.getParameterCount()) { @@ -82,6 +76,8 @@ public R invoke(Object target, Object... args) { return (R) newInstance(args); } + /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + @Deprecated // will become package-private @Override @SuppressWarnings("unchecked") public R invokeChecked(Object target, Object... args) throws Exception { @@ -172,16 +168,6 @@ public Builder impl(Class targetClass, Class... types) { return this; } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; This varargs method conflicts with {@link - * #hiddenImpl(Class, Class...)}. Use {@link #builder(Class)} instead. - */ - @Deprecated - public Builder hiddenImpl(Class... types) { - hiddenImpl(baseClass, types); - return this; - } - public Builder hiddenImpl(String className, Class... types) { // don't do any work if an implementation has been found if (ctor != null) { diff --git a/common/src/main/java/org/apache/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java index e88affa0cbdd..cc397d329e94 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynFields.java +++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java @@ -386,21 +386,6 @@ public BoundField build(Object target) { return this.build().bind(target); } - /** - * Returns the first valid implementation as a StaticField or throws a NoSuchFieldException if - * there is none. - * - * @param Java class stored in the field - * @return a {@link StaticField} with a valid implementation - * @throws IllegalStateException if the method is not static - * @throws NoSuchFieldException if no implementation was found - * @deprecated since 1.6.0, will be removed in 1.7.0 - */ - @Deprecated - public StaticField buildStaticChecked() throws NoSuchFieldException { - return this.buildChecked().asStatic(); - } - /** * Returns the first valid implementation as a StaticField or throws a RuntimeException if there * is none. diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index fc0e578c7dd1..65a69bd0e12c 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -51,10 +51,8 @@ public static class UnboundMethod { (method == null || method.isVarArgs()) ? -1 : method.getParameterTypes().length; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated // will become private @SuppressWarnings("unchecked") - public R invokeChecked(Object target, Object... args) throws Exception { + R invokeChecked(Object target, Object... args) throws Exception { try { if (argLength < 0) { return (R) method.invoke(target, args); @@ -127,6 +125,8 @@ public String toString() { /** Singleton {@link UnboundMethod}, performs no operation and returns null. */ private static final UnboundMethod NOOP = new UnboundMethod(null, "NOOP") { + /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + @Deprecated // will become package-private @Override public R invokeChecked(Object target, Object... args) { return null; @@ -315,38 +315,6 @@ public Builder impl(Class targetClass, Class... argClasses) { return this; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Builder ctorImpl(Class targetClass, Class... argClasses) { - // don't do any work if an implementation has been found - if (method != null) { - return this; - } - - try { - this.method = new DynConstructors.Builder().impl(targetClass, argClasses).buildChecked(); - } catch (NoSuchMethodException e) { - // not the right implementation - } - return this; - } - - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Builder ctorImpl(String className, Class... argClasses) { - // don't do any work if an implementation has been found - if (method != null) { - return this; - } - - try { - this.method = new DynConstructors.Builder().impl(className, argClasses).buildChecked(); - } catch (NoSuchMethodException e) { - // not the right implementation - } - return this; - } - /** * Checks for an implementation, first finding the given class by name. * diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index f4019d688cb8..526bb42ea687 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -262,7 +262,7 @@ private boolean fileContent(BoundReference ref) { return ref.fieldId() == DataFile.CONTENT.fieldId(); } - private boolean contentMatch(Integer fileContentId) { + private boolean contentMatch(Integer fileContentId) { if (FileContent.DATA.id() == fileContentId) { return ManifestContent.DATA.id() == manifestContentId; } else if (FileContent.EQUALITY_DELETES.id() == fileContentId diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 90d435811020..5c82bc877a15 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -284,17 +284,6 @@ public long newSnapshotId() { }; } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link - * BaseMetastoreOperations.CommitStatus} instead - */ - @Deprecated - protected enum CommitStatus { - FAILURE, - SUCCESS, - UNKNOWN - } - /** * Attempt to load the table and see if any current or past metadata location matches the one we * were attempting to set. This is used as a last resort when we are dealing with exceptions that diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index dce6d4a995bd..985fb7944087 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -190,6 +190,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List apply = Lists.newArrayList(); Iterables.addAll(apply, newManifestsWithMetadata); apply.addAll(keptManifests); + apply.forEach(summaryBuilder::addedManifestStats); return apply; } diff --git a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java index b59292c397a1..c69f6f3844f9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java +++ b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java @@ -59,11 +59,13 @@ class BaseUpdatePartitionSpec implements UpdatePartitionSpec { private final Map renames = Maps.newHashMap(); private boolean caseSensitive; + private boolean setAsDefault; private int lastAssignedPartitionId; BaseUpdatePartitionSpec(TableOperations ops) { this.ops = ops; this.caseSensitive = true; + this.setAsDefault = true; this.base = ops.current(); this.formatVersion = base.formatVersion(); this.spec = base.spec(); @@ -95,6 +97,7 @@ class BaseUpdatePartitionSpec implements UpdatePartitionSpec { this.base = null; this.formatVersion = formatVersion; this.caseSensitive = true; + this.setAsDefault = true; this.spec = spec; this.schema = spec.schema(); this.nameToField = indexSpecByName(spec); @@ -118,7 +121,7 @@ private int assignFieldId() { */ private PartitionField recycleOrCreatePartitionField( Pair> sourceTransform, String name) { - if (formatVersion == 2 && base != null) { + if (formatVersion >= 2 && base != null) { int sourceId = sourceTransform.first(); Transform transform = sourceTransform.second(); @@ -146,6 +149,12 @@ public UpdatePartitionSpec caseSensitive(boolean isCaseSensitive) { return this; } + @Override + public UpdatePartitionSpec addNonDefaultSpec() { + this.setAsDefault = false; + return this; + } + @Override public BaseUpdatePartitionSpec addField(String sourceName) { return addField(Expressions.ref(sourceName)); @@ -327,7 +336,12 @@ public PartitionSpec apply() { @Override public void commit() { - TableMetadata update = base.updatePartitionSpec(apply()); + TableMetadata update; + if (setAsDefault) { + update = base.updatePartitionSpec(apply()); + } else { + update = base.addPartitionSpec(apply()); + } ops.commit(base, update); } diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1439289130d7..1c574d2f07b7 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -162,6 +162,8 @@ public List apply(TableMetadata base, Snapshot snapshot) { manifests.addAll(snapshot.allManifests(ops.io())); } + manifests.forEach(summaryBuilder::addedManifestStats); + return manifests; } @@ -198,6 +200,16 @@ protected void cleanUncommitted(Set committed) { } } + /** + * Cleanup after committing is disabled for FastAppend unless there are rewrittenAppendManifests + * because: 1.) Appended manifests are never rewritten 2.) Manifests which are written out as part + * of appendFile are already cleaned up between commit attempts in writeNewManifests + */ + @Override + protected boolean cleanupAfterCommit() { + return !rewrittenAppendManifests.isEmpty(); + } + private List writeNewManifests() throws IOException { if (hasNewFiles && newManifests != null) { newManifests.forEach(file -> deleteFile(file.path())); diff --git a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java index a6ea41319f4e..7ae7dc74a72e 100644 --- a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java +++ b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java @@ -40,35 +40,6 @@ public class FileScanTaskParser { private FileScanTaskParser() {} - /** - * Serialize file scan task to JSON string - * - * @deprecated will be removed in 1.7.0; use {@link ScanTaskParser#toJson(FileScanTask)} instead - */ - @Deprecated - public static String toJson(FileScanTask fileScanTask) { - Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); - return JsonUtil.generate( - generator -> { - generator.writeStartObject(); - toJson(fileScanTask, generator); - generator.writeEndObject(); - }, - false); - } - - /** - * Deserialize file scan task from JSON string - * - * @deprecated will be removed in 1.7.0; use {@link ScanTaskParser#fromJson(String, boolean)} - * instead - */ - @Deprecated - public static FileScanTask fromJson(String json, boolean caseSensitive) { - Preconditions.checkArgument(json != null, "Invalid JSON string for file scan task: null"); - return JsonUtil.parse(json, node -> fromJson(node, caseSensitive)); - } - static void toJson(FileScanTask fileScanTask, JsonGenerator generator) throws IOException { Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 840c90bebdde..f630bb3eb743 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -182,6 +182,8 @@ public static ManifestWriter write( return new ManifestWriter.V1Writer(spec, encryptedOutputFile, snapshotId); case 2: return new ManifestWriter.V2Writer(spec, encryptedOutputFile, snapshotId); + case 3: + return new ManifestWriter.V3Writer(spec, encryptedOutputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -238,6 +240,8 @@ public static ManifestWriter writeDeleteManifest( throw new IllegalArgumentException("Cannot write delete files in a v1 table"); case 2: return new ManifestWriter.V2DeleteWriter(spec, outputFile, snapshotId); + case 3: + return new ManifestWriter.V3DeleteWriter(spec, outputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java index 3f7f20d4df6c..b17eedad18af 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -70,6 +70,41 @@ public long length() { return writer.length(); } + static class V3Writer extends ManifestListWriter { + private final V3Metadata.IndexedManifestFile wrapper; + + V3Writer(OutputFile snapshotFile, long snapshotId, Long parentSnapshotId, long sequenceNumber) { + super( + snapshotFile, + ImmutableMap.of( + "snapshot-id", String.valueOf(snapshotId), + "parent-snapshot-id", String.valueOf(parentSnapshotId), + "sequence-number", String.valueOf(sequenceNumber), + "format-version", "3")); + this.wrapper = new V3Metadata.IndexedManifestFile(snapshotId, sequenceNumber); + } + + @Override + protected ManifestFile prepare(ManifestFile manifest) { + return wrapper.wrap(manifest); + } + + @Override + protected FileAppender newAppender(OutputFile file, Map meta) { + try { + return Avro.write(file) + .schema(V3Metadata.MANIFEST_LIST_SCHEMA) + .named("manifest_file") + .meta(meta) + .overwrite() + .build(); + + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create snapshot list writer for path: %s", file); + } + } + } + static class V2Writer extends ManifestListWriter { private final V2Metadata.IndexedManifestFile wrapper; diff --git a/core/src/main/java/org/apache/iceberg/ManifestLists.java b/core/src/main/java/org/apache/iceberg/ManifestLists.java index c7b3e5fee5a9..f20a481cf25a 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestLists.java +++ b/core/src/main/java/org/apache/iceberg/ManifestLists.java @@ -66,6 +66,9 @@ static ManifestListWriter write( case 2: return new ManifestListWriter.V2Writer( manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); + case 3: + return new ManifestListWriter.V3Writer( + manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); } throw new UnsupportedOperationException( "Cannot write manifest list for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index cea907ddaca1..88587a1ebc89 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -217,6 +217,79 @@ public void close() throws IOException { writer.close(); } + static class V3Writer extends ManifestWriter { + private final V3Metadata.IndexedManifestEntry entryWrapper; + + V3Writer(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry prepare(ManifestEntry entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender> newAppender( + PartitionSpec spec, OutputFile file) { + Schema manifestSchema = V3Metadata.entrySchema(spec.partitionType()); + try { + return Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "3") + .meta("content", "data") + .overwrite() + .build(); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); + } + } + } + + static class V3DeleteWriter extends ManifestWriter { + private final V3Metadata.IndexedManifestEntry entryWrapper; + + V3DeleteWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry prepare(ManifestEntry entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender> newAppender( + PartitionSpec spec, OutputFile file) { + Schema manifestSchema = V3Metadata.entrySchema(spec.partitionType()); + try { + return Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "3") + .meta("content", "deletes") + .overwrite() + .build(); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); + } + } + + @Override + protected ManifestContent content() { + return ManifestContent.DELETES; + } + } + static class V2Writer extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index b4c0567ab73a..cedc62294c1d 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -869,6 +869,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List manifests = Lists.newArrayList(); Iterables.addAll(manifests, mergeManager.mergeManifests(unmergedManifests)); Iterables.addAll(manifests, deleteMergeManager.mergeManifests(unmergedDeleteManifests)); + manifests.forEach(summaryBuilder::addedManifestStats); return manifests; } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 9f4bcbc6bba9..74997cc89849 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -41,7 +41,7 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptingFileIO; @@ -368,8 +368,8 @@ protected TableMetadata refresh() { @Override @SuppressWarnings("checkstyle:CyclomaticComplexity") public void commit() { - // this is always set to the latest commit attempt's snapshot id. - AtomicLong newSnapshotId = new AtomicLong(-1L); + // this is always set to the latest commit attempt's snapshot + AtomicReference stagedSnapshot = new AtomicReference<>(); try (Timed ignore = commitMetrics().totalDuration().start()) { try { Tasks.foreach(ops) @@ -384,7 +384,7 @@ public void commit() { .run( taskOps -> { Snapshot newSnapshot = apply(); - newSnapshotId.set(newSnapshot.snapshotId()); + stagedSnapshot.set(newSnapshot); TableMetadata.Builder update = TableMetadata.buildFrom(base); if (base.snapshot(newSnapshot.snapshotId()) != null) { // this is a rollback operation @@ -422,26 +422,23 @@ public void commit() { throw e; } + // at this point, the commit must have succeeded so the stagedSnapshot is committed + Snapshot committedSnapshot = stagedSnapshot.get(); try { - LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName()); - - // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by - // id in case another commit was added between this commit and the refresh. - Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); - if (saved != null) { - cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io()))); - // also clean up unused manifest lists created by multiple attempts - for (String manifestList : manifestLists) { - if (!saved.manifestListLocation().equals(manifestList)) { - deleteFile(manifestList); - } + LOG.info( + "Committed snapshot {} ({})", + committedSnapshot.snapshotId(), + getClass().getSimpleName()); + + if (cleanupAfterCommit()) { + cleanUncommitted(Sets.newHashSet(committedSnapshot.allManifests(ops.io()))); + } + // also clean up unused manifest lists created by multiple attempts + for (String manifestList : manifestLists) { + if (!committedSnapshot.manifestListLocation().equals(manifestList)) { + deleteFile(manifestList); } - } else { - // saved may not be present if the latest metadata couldn't be loaded due to eventual - // consistency problems in refresh. in that case, don't clean up. - LOG.warn("Failed to load committed snapshot, skipping manifest clean-up"); } - } catch (Throwable e) { LOG.warn( "Failed to load committed table metadata or during cleanup, skipping further cleanup", @@ -503,18 +500,6 @@ protected OutputFile manifestListPath() { "snap-%d-%d-%s", snapshotId(), attempt.incrementAndGet(), commitUUID)))); } - /** - * @deprecated will be removed in 1.7.0; Use {@link SnapshotProducer#newManifestOutputFile} - * instead - */ - @Deprecated - protected OutputFile newManifestOutput() { - return ops.io() - .newOutputFile( - ops.metadataFileLocation( - FileFormat.AVRO.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement()))); - } - protected EncryptedOutputFile newManifestOutputFile() { String manifestFileLocation = ops.metadataFileLocation( @@ -565,6 +550,10 @@ protected boolean canInheritSnapshotId() { return canInheritSnapshotId; } + protected boolean cleanupAfterCommit() { + return true; + } + private static ManifestFile addMetadata(TableOperations ops, ManifestFile manifest) { try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById())) { diff --git a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java index 22c9df2a8eaf..7ca4a6ff14fa 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotSummary.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotSummary.java @@ -58,6 +58,8 @@ public class SnapshotSummary { public static final String SOURCE_SNAPSHOT_ID_PROP = "source-snapshot-id"; public static final String REPLACE_PARTITIONS_PROP = "replace-partitions"; public static final String EXTRA_METADATA_PREFIX = "snapshot-property."; + public static final String TOTAL_DATA_MANIFEST_FILES = "total-data-manifest-files"; + public static final String TOTAL_DELETE_MANIFEST_FILES = "total-delete-manifest-files"; public static final MapJoiner MAP_JOINER = Joiner.on(",").withKeyValueSeparator("="); @@ -144,6 +146,10 @@ public void addedManifest(ManifestFile manifest) { metrics.addedManifest(manifest); } + public void addedManifestStats(ManifestFile manifest) { + metrics.addedManifestStats(manifest); + } + public void set(String property, String value) { properties.put(property, value); } @@ -229,6 +235,8 @@ private static class UpdateMetrics { private long removedPosDeletes = 0L; private long addedEqDeletes = 0L; private long removedEqDeletes = 0L; + private long totalDataManifestFiles = 0L; + private long totalDeleteManifestFiles = 0L; private boolean trustSizeAndDeleteCounts = true; void clear() { @@ -248,6 +256,8 @@ void clear() { this.removedPosDeletes = 0L; this.addedEqDeletes = 0L; this.removedEqDeletes = 0L; + this.totalDataManifestFiles = 0L; + this.totalDeleteManifestFiles = 0L; this.trustSizeAndDeleteCounts = true; } @@ -263,6 +273,12 @@ void addTo(ImmutableMap.Builder builder) { setIf(removedDeleteFiles > 0, builder, REMOVED_DELETE_FILES_PROP, removedDeleteFiles); setIf(addedRecords > 0, builder, ADDED_RECORDS_PROP, addedRecords); setIf(deletedRecords > 0, builder, DELETED_RECORDS_PROP, deletedRecords); + setIf(totalDataManifestFiles > 0, builder, TOTAL_DATA_MANIFEST_FILES, totalDataManifestFiles); + setIf( + totalDeleteManifestFiles > 0, + builder, + TOTAL_DELETE_MANIFEST_FILES, + totalDeleteManifestFiles); if (trustSizeAndDeleteCounts) { setIf(addedSize > 0, builder, ADDED_FILE_SIZE_PROP, addedSize); @@ -336,6 +352,16 @@ void addedManifest(ManifestFile manifest) { } } + void addedManifestStats(ManifestFile manifest) { + switch (manifest.content()) { + case DATA: + this.totalDataManifestFiles++; + break; + case DELETES: + this.totalDeleteManifestFiles++; + } + } + void merge(UpdateMetrics other) { this.addedFiles += other.addedFiles; this.removedFiles += other.removedFiles; @@ -353,6 +379,8 @@ void merge(UpdateMetrics other) { this.removedPosDeletes += other.removedPosDeletes; this.addedEqDeletes += other.addedEqDeletes; this.removedEqDeletes += other.removedEqDeletes; + this.totalDataManifestFiles += other.totalDataManifestFiles; + this.totalDeleteManifestFiles += other.totalDeleteManifestFiles; this.trustSizeAndDeleteCounts = trustSizeAndDeleteCounts && other.trustSizeAndDeleteCounts; } } diff --git a/core/src/main/java/org/apache/iceberg/StaticDataTask.java b/core/src/main/java/org/apache/iceberg/StaticDataTask.java index f25ebd49c9d8..1a396f0bfc7e 100644 --- a/core/src/main/java/org/apache/iceberg/StaticDataTask.java +++ b/core/src/main/java/org/apache/iceberg/StaticDataTask.java @@ -127,7 +127,11 @@ DataFile metadataFile() { return metadataFile; } - /** @return the table rows before projection */ + /** + * Returns the table rows before projection. + * + * @return the table rows before projection + */ StructLike[] tableRows() { return rows; } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 74b8ad0bbddc..923db6bbd68f 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -51,7 +51,7 @@ public class TableMetadata implements Serializable { static final long INITIAL_SEQUENCE_NUMBER = 0; static final long INVALID_SEQUENCE_NUMBER = -1; static final int DEFAULT_TABLE_FORMAT_VERSION = 2; - static final int SUPPORTED_TABLE_FORMAT_VERSION = 2; + static final int SUPPORTED_TABLE_FORMAT_VERSION = 3; static final int INITIAL_SPEC_ID = 0; static final int INITIAL_SORT_ORDER_ID = 1; static final int INITIAL_SCHEMA_ID = 0; @@ -564,6 +564,10 @@ public TableMetadata updatePartitionSpec(PartitionSpec newPartitionSpec) { return new Builder(this).setDefaultPartitionSpec(newPartitionSpec).build(); } + public TableMetadata addPartitionSpec(PartitionSpec newPartitionSpec) { + return new Builder(this).addPartitionSpec(newPartitionSpec).build(); + } + public TableMetadata replaceSortOrder(SortOrder newOrder) { return new Builder(this).setDefaultSortOrder(newOrder).build(); } diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java new file mode 100644 index 000000000000..94e20ea99858 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -0,0 +1,575 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +class V3Metadata { + private V3Metadata() {} + + static final Schema MANIFEST_LIST_SCHEMA = + new Schema( + ManifestFile.PATH, + ManifestFile.LENGTH, + ManifestFile.SPEC_ID, + ManifestFile.MANIFEST_CONTENT.asRequired(), + ManifestFile.SEQUENCE_NUMBER.asRequired(), + ManifestFile.MIN_SEQUENCE_NUMBER.asRequired(), + ManifestFile.SNAPSHOT_ID.asRequired(), + ManifestFile.ADDED_FILES_COUNT.asRequired(), + ManifestFile.EXISTING_FILES_COUNT.asRequired(), + ManifestFile.DELETED_FILES_COUNT.asRequired(), + ManifestFile.ADDED_ROWS_COUNT.asRequired(), + ManifestFile.EXISTING_ROWS_COUNT.asRequired(), + ManifestFile.DELETED_ROWS_COUNT.asRequired(), + ManifestFile.PARTITION_SUMMARIES, + ManifestFile.KEY_METADATA); + + /** + * A wrapper class to write any ManifestFile implementation to Avro using the v3 write schema. + * + *

This is used to maintain compatibility with v3 by writing manifest list files with the old + * schema, instead of writing a sequence number into metadata files in v3 tables. + */ + static class IndexedManifestFile implements ManifestFile, IndexedRecord { + private static final org.apache.avro.Schema AVRO_SCHEMA = + AvroSchemaUtil.convert(MANIFEST_LIST_SCHEMA, "manifest_file"); + + private final long commitSnapshotId; + private final long sequenceNumber; + private ManifestFile wrapped = null; + + IndexedManifestFile(long commitSnapshotId, long sequenceNumber) { + this.commitSnapshotId = commitSnapshotId; + this.sequenceNumber = sequenceNumber; + } + + public ManifestFile wrap(ManifestFile file) { + this.wrapped = file; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return AVRO_SCHEMA; + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedManifestFile wrapper via put"); + } + + @Override + public Object get(int pos) { + switch (pos) { + case 0: + return wrapped.path(); + case 1: + return wrapped.length(); + case 2: + return wrapped.partitionSpecId(); + case 3: + return wrapped.content().id(); + case 4: + if (wrapped.sequenceNumber() == ManifestWriter.UNASSIGNED_SEQ) { + // if the sequence number is being assigned here, then the manifest must be created by + // the current + // operation. to validate this, check that the snapshot id matches the current commit + Preconditions.checkState( + commitSnapshotId == wrapped.snapshotId(), + "Found unassigned sequence number for a manifest from snapshot: %s", + wrapped.snapshotId()); + return sequenceNumber; + } else { + return wrapped.sequenceNumber(); + } + case 5: + if (wrapped.minSequenceNumber() == ManifestWriter.UNASSIGNED_SEQ) { + // same sanity check as above + Preconditions.checkState( + commitSnapshotId == wrapped.snapshotId(), + "Found unassigned sequence number for a manifest from snapshot: %s", + wrapped.snapshotId()); + // if the min sequence number is not determined, then there was no assigned sequence + // number for any file + // written to the wrapped manifest. replace the unassigned sequence number with the one + // for this commit + return sequenceNumber; + } else { + return wrapped.minSequenceNumber(); + } + case 6: + return wrapped.snapshotId(); + case 7: + return wrapped.addedFilesCount(); + case 8: + return wrapped.existingFilesCount(); + case 9: + return wrapped.deletedFilesCount(); + case 10: + return wrapped.addedRowsCount(); + case 11: + return wrapped.existingRowsCount(); + case 12: + return wrapped.deletedRowsCount(); + case 13: + return wrapped.partitions(); + case 14: + return wrapped.keyMetadata(); + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + } + } + + @Override + public String path() { + return wrapped.path(); + } + + @Override + public long length() { + return wrapped.length(); + } + + @Override + public int partitionSpecId() { + return wrapped.partitionSpecId(); + } + + @Override + public ManifestContent content() { + return wrapped.content(); + } + + @Override + public long sequenceNumber() { + return wrapped.sequenceNumber(); + } + + @Override + public long minSequenceNumber() { + return wrapped.minSequenceNumber(); + } + + @Override + public Long snapshotId() { + return wrapped.snapshotId(); + } + + @Override + public boolean hasAddedFiles() { + return wrapped.hasAddedFiles(); + } + + @Override + public Integer addedFilesCount() { + return wrapped.addedFilesCount(); + } + + @Override + public Long addedRowsCount() { + return wrapped.addedRowsCount(); + } + + @Override + public boolean hasExistingFiles() { + return wrapped.hasExistingFiles(); + } + + @Override + public Integer existingFilesCount() { + return wrapped.existingFilesCount(); + } + + @Override + public Long existingRowsCount() { + return wrapped.existingRowsCount(); + } + + @Override + public boolean hasDeletedFiles() { + return wrapped.hasDeletedFiles(); + } + + @Override + public Integer deletedFilesCount() { + return wrapped.deletedFilesCount(); + } + + @Override + public Long deletedRowsCount() { + return wrapped.deletedRowsCount(); + } + + @Override + public List partitions() { + return wrapped.partitions(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public ManifestFile copy() { + return wrapped.copy(); + } + } + + static Schema entrySchema(Types.StructType partitionType) { + return wrapFileSchema(fileType(partitionType)); + } + + static Schema wrapFileSchema(Types.StructType fileSchema) { + // this is used to build projection schemas + return new Schema( + ManifestEntry.STATUS, + ManifestEntry.SNAPSHOT_ID, + ManifestEntry.SEQUENCE_NUMBER, + ManifestEntry.FILE_SEQUENCE_NUMBER, + required(ManifestEntry.DATA_FILE_ID, "data_file", fileSchema)); + } + + static Types.StructType fileType(Types.StructType partitionType) { + return Types.StructType.of( + DataFile.CONTENT.asRequired(), + DataFile.FILE_PATH, + DataFile.FILE_FORMAT, + required( + DataFile.PARTITION_ID, DataFile.PARTITION_NAME, partitionType, DataFile.PARTITION_DOC), + DataFile.RECORD_COUNT, + DataFile.FILE_SIZE, + DataFile.COLUMN_SIZES, + DataFile.VALUE_COUNTS, + DataFile.NULL_VALUE_COUNTS, + DataFile.NAN_VALUE_COUNTS, + DataFile.LOWER_BOUNDS, + DataFile.UPPER_BOUNDS, + DataFile.KEY_METADATA, + DataFile.SPLIT_OFFSETS, + DataFile.EQUALITY_IDS, + DataFile.SORT_ORDER_ID); + } + + static class IndexedManifestEntry> + implements ManifestEntry, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final Long commitSnapshotId; + private final IndexedDataFile fileWrapper; + private ManifestEntry wrapped = null; + + IndexedManifestEntry(Long commitSnapshotId, Types.StructType partitionType) { + this.avroSchema = AvroSchemaUtil.convert(entrySchema(partitionType), "manifest_entry"); + this.commitSnapshotId = commitSnapshotId; + this.fileWrapper = new IndexedDataFile<>(partitionType); + } + + public IndexedManifestEntry wrap(ManifestEntry entry) { + this.wrapped = entry; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedManifestEntry wrapper via put"); + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return wrapped.status().id(); + case 1: + return wrapped.snapshotId(); + case 2: + if (wrapped.dataSequenceNumber() == null) { + // if the entry's data sequence number is null, + // then it will inherit the sequence number of the current commit. + // to validate that this is correct, check that the snapshot id is either null (will + // also be inherited) or that it matches the id of the current commit. + Preconditions.checkState( + wrapped.snapshotId() == null || wrapped.snapshotId().equals(commitSnapshotId), + "Found unassigned sequence number for an entry from snapshot: %s", + wrapped.snapshotId()); + + // inheritance should work only for ADDED entries + Preconditions.checkState( + wrapped.status() == Status.ADDED, + "Only entries with status ADDED can have null sequence number"); + + return null; + } + return wrapped.dataSequenceNumber(); + case 3: + return wrapped.fileSequenceNumber(); + case 4: + return fileWrapper.wrap(wrapped.file()); + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public Status status() { + return wrapped.status(); + } + + @Override + public Long snapshotId() { + return wrapped.snapshotId(); + } + + @Override + public void setSnapshotId(long snapshotId) { + wrapped.setSnapshotId(snapshotId); + } + + @Override + public Long dataSequenceNumber() { + return wrapped.dataSequenceNumber(); + } + + @Override + public void setDataSequenceNumber(long dataSequenceNumber) { + wrapped.setDataSequenceNumber(dataSequenceNumber); + } + + @Override + public Long fileSequenceNumber() { + return wrapped.fileSequenceNumber(); + } + + @Override + public void setFileSequenceNumber(long fileSequenceNumber) { + wrapped.setFileSequenceNumber(fileSequenceNumber); + } + + @Override + public F file() { + return wrapped.file(); + } + + @Override + public ManifestEntry copy() { + return wrapped.copy(); + } + + @Override + public ManifestEntry copyWithoutStats() { + return wrapped.copyWithoutStats(); + } + } + + /** Wrapper used to write DataFile or DeleteFile to v3 metadata. */ + static class IndexedDataFile implements ContentFile, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final IndexedStructLike partitionWrapper; + private ContentFile wrapped = null; + + IndexedDataFile(Types.StructType partitionType) { + this.avroSchema = AvroSchemaUtil.convert(fileType(partitionType), "data_file"); + this.partitionWrapper = new IndexedStructLike(avroSchema.getField("partition").schema()); + } + + @SuppressWarnings("unchecked") + IndexedDataFile wrap(ContentFile file) { + this.wrapped = (ContentFile) file; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } + + @Override + public Object get(int pos) { + switch (pos) { + case 0: + return wrapped.content().id(); + case 1: + return wrapped.path().toString(); + case 2: + return wrapped.format() != null ? wrapped.format().toString() : null; + case 3: + return partitionWrapper.wrap(wrapped.partition()); + case 4: + return wrapped.recordCount(); + case 5: + return wrapped.fileSizeInBytes(); + case 6: + return wrapped.columnSizes(); + case 7: + return wrapped.valueCounts(); + case 8: + return wrapped.nullValueCounts(); + case 9: + return wrapped.nanValueCounts(); + case 10: + return wrapped.lowerBounds(); + case 11: + return wrapped.upperBounds(); + case 12: + return wrapped.keyMetadata(); + case 13: + return wrapped.splitOffsets(); + case 14: + return wrapped.equalityFieldIds(); + case 15: + return wrapped.sortOrderId(); + } + throw new IllegalArgumentException("Unknown field ordinal: " + pos); + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedDataFile wrapper via put"); + } + + @Override + public Long pos() { + return null; + } + + @Override + public int specId() { + return wrapped.specId(); + } + + @Override + public FileContent content() { + return wrapped.content(); + } + + @Override + public CharSequence path() { + return wrapped.path(); + } + + @Override + public FileFormat format() { + return wrapped.format(); + } + + @Override + public StructLike partition() { + return wrapped.partition(); + } + + @Override + public long recordCount() { + return wrapped.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return wrapped.fileSizeInBytes(); + } + + @Override + public Map columnSizes() { + return wrapped.columnSizes(); + } + + @Override + public Map valueCounts() { + return wrapped.valueCounts(); + } + + @Override + public Map nullValueCounts() { + return wrapped.nullValueCounts(); + } + + @Override + public Map nanValueCounts() { + return wrapped.nanValueCounts(); + } + + @Override + public Map lowerBounds() { + return wrapped.lowerBounds(); + } + + @Override + public Map upperBounds() { + return wrapped.upperBounds(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public List splitOffsets() { + return wrapped.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return wrapped.equalityFieldIds(); + } + + @Override + public Integer sortOrderId() { + return wrapped.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + return wrapped.dataSequenceNumber(); + } + + @Override + public Long fileSequenceNumber() { + return wrapped.fileSequenceNumber(); + } + + @Override + public F copy() { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + + @Override + public F copyWithStats(Set requestedColumnIds) { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + + @Override + public F copyWithoutStats() { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java index fb3c27220cb2..cea7003c1a38 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java @@ -229,7 +229,8 @@ protected long numOutputFiles(long inputSize) { // the remainder file is of a valid size for this rewrite so keep it return numFilesWithRemainder; - } else if (avgFileSizeWithoutRemainder < Math.min(1.1 * targetFileSize, writeMaxFileSize())) { + } else if (avgFileSizeWithoutRemainder + < Math.min(1.1 * targetFileSize, (double) writeMaxFileSize())) { // if the reminder is distributed amongst other files, // the average file size will be no more than 10% bigger than the target file size // so round down and distribute remainder amongst other files diff --git a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java index ff20ba53ff70..cef57cd16726 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -36,6 +36,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.Filter; @@ -398,33 +399,9 @@ private static class DataFileFilter extends Filter { @Override protected boolean shouldKeep(T posDelete) { - return charSeqEquals(dataLocation, (CharSequence) FILENAME_ACCESSOR.get(posDelete)); - } - - private boolean charSeqEquals(CharSequence s1, CharSequence s2) { - if (s1 == s2) { - return true; - } - - int count = s1.length(); - if (count != s2.length()) { - return false; - } - - if (s1 instanceof String && s2 instanceof String && s1.hashCode() != s2.hashCode()) { - return false; - } - - // File paths inside a delete file normally have more identical chars at the beginning. For - // example, a typical - // path is like "s3:/bucket/db/table/data/partition/00000-0-[uuid]-00001.parquet". - // The uuid is where the difference starts. So it's faster to find the first diff backward. - for (int i = count - 1; i >= 0; i--) { - if (s1.charAt(i) != s2.charAt(i)) { - return false; - } - } - return true; + return Comparators.filePath() + .compare(dataLocation, (CharSequence) FILENAME_ACCESSOR.get(posDelete)) + == 0; } } } diff --git a/core/src/main/java/org/apache/iceberg/io/ContentCache.java b/core/src/main/java/org/apache/iceberg/io/ContentCache.java index 7942c69d5d77..ce37cfb08934 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -28,7 +28,6 @@ import java.nio.ByteBuffer; import java.time.Duration; import java.util.List; -import java.util.function.Function; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -111,24 +110,6 @@ public CacheStats stats() { return cache.stats(); } - /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ - @Deprecated - public CacheEntry get(String key, Function mappingFunction) { - return cache.get(key, mappingFunction); - } - - /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ - @Deprecated - public CacheEntry getIfPresent(String location) { - return cache.getIfPresent(location); - } - - /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ - @Deprecated - public InputFile tryCache(FileIO io, String location, long length) { - return tryCache(io.newInputFile(location, length)); - } - /** * Try cache the file-content of file in the given location upon stream reading. * @@ -173,11 +154,7 @@ public String toString() { .toString(); } - /** @deprecated will be removed in 1.7; use {@link FileContent} instead. */ - @Deprecated - private static class CacheEntry {} - - private static class FileContent extends CacheEntry { + private static class FileContent { private final long length; private final List buffers; diff --git a/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java b/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java index ad66e8d32408..e5e4972603dc 100644 --- a/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java +++ b/core/src/main/java/org/apache/iceberg/metrics/CommitMetricsResult.java @@ -50,6 +50,8 @@ public interface CommitMetricsResult { String ADDED_EQ_DELETES = "added-equality-deletes"; String REMOVED_EQ_DELETES = "removed-equality-deletes"; String TOTAL_EQ_DELETES = "total-equality-deletes"; + String TOTAL_DATA_MANIFEST_FILES = "total-data-manifest-files"; + String TOTAL_DELETE_MANIFEST_FILES = "total-delete-manifest-files"; @Nullable TimerResult totalDuration(); @@ -123,6 +125,12 @@ public interface CommitMetricsResult { @Nullable CounterResult totalEqualityDeletes(); + @Nullable + CounterResult totalDataManifestFiles(); + + @Nullable + CounterResult totalDeleteManifestFiles(); + static CommitMetricsResult from( CommitMetrics commitMetrics, Map snapshotSummary) { Preconditions.checkArgument(null != commitMetrics, "Invalid commit metrics: null"); @@ -163,6 +171,10 @@ static CommitMetricsResult from( .removedEqualityDeletes( counterFrom(snapshotSummary, SnapshotSummary.REMOVED_EQ_DELETES_PROP)) .totalEqualityDeletes(counterFrom(snapshotSummary, SnapshotSummary.TOTAL_EQ_DELETES_PROP)) + .totalDataManifestFiles( + counterFrom(snapshotSummary, SnapshotSummary.TOTAL_DATA_MANIFEST_FILES)) + .totalDeleteManifestFiles( + counterFrom(snapshotSummary, SnapshotSummary.TOTAL_DELETE_MANIFEST_FILES)) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java b/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java index aadb97bc7112..263b3c305af0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java +++ b/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java @@ -149,7 +149,7 @@ public TimeValue getRetryInterval(HttpResponse response, int execCount, HttpCont } } - int delayMillis = 1000 * (int) Math.min(Math.pow(2.0, (long) execCount - 1), 64.0); + int delayMillis = 1000 * (int) Math.min(Math.pow(2.0, (long) execCount - 1.0), 64.0); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMillis * 0.1))); return TimeValue.ofMilliseconds(delayMillis + jitter); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index a72d3958c140..1c607e3b0220 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -547,7 +547,7 @@ public void createNamespace( public List listNamespaces(SessionContext context, Namespace namespace) { Map queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { - queryParams.put("parent", RESTUtil.NAMESPACE_JOINER.join(namespace.levels())); + queryParams.put("parent", RESTUtil.encodeNamespace(namespace)); } ImmutableList.Builder namespaces = ImmutableList.builder(); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java index fab01162cad7..45422b8ae8b5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java @@ -33,14 +33,24 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; public class RESTUtil { - private static final char NAMESPACE_SEPARATOR = '\u001f'; - public static final Joiner NAMESPACE_JOINER = Joiner.on(NAMESPACE_SEPARATOR); - public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_SEPARATOR); private static final String NAMESPACE_ESCAPED_SEPARATOR = "%1F"; private static final Joiner NAMESPACE_ESCAPED_JOINER = Joiner.on(NAMESPACE_ESCAPED_SEPARATOR); private static final Splitter NAMESPACE_ESCAPED_SPLITTER = Splitter.on(NAMESPACE_ESCAPED_SEPARATOR); + /** + * @deprecated since 1.7.0, will be made private in 1.8.0; use {@link + * RESTUtil#encodeNamespace(Namespace)} instead. + */ + @Deprecated public static final Joiner NAMESPACE_JOINER = Joiner.on(NAMESPACE_ESCAPED_SEPARATOR); + + /** + * @deprecated since 1.7.0, will be made private in 1.8.0; use {@link + * RESTUtil#decodeNamespace(String)} instead. + */ + @Deprecated + public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_ESCAPED_SEPARATOR); + private RESTUtil() {} public static String stripTrailingSlash(String path) { diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 189e5fde2cad..52c89af9d474 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -465,26 +465,6 @@ public AuthSession(Map baseHeaders, AuthConfig config) { this.config = config; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public AuthSession( - Map baseHeaders, - String token, - String tokenType, - String credential, - String scope, - String oauth2ServerUri) { - this( - baseHeaders, - AuthConfig.builder() - .token(token) - .tokenType(tokenType) - .credential(credential) - .scope(scope) - .oauth2ServerUri(oauth2ServerUri) - .build()); - } - public Map headers() { return headers; } diff --git a/core/src/main/java/org/apache/iceberg/util/Pair.java b/core/src/main/java/org/apache/iceberg/util/Pair.java index bd3a934f6f04..e36321c8e2c9 100644 --- a/core/src/main/java/org/apache/iceberg/util/Pair.java +++ b/core/src/main/java/org/apache/iceberg/util/Pair.java @@ -58,11 +58,6 @@ public Schema load(Pair, Class> key) { private X first; private Y second; - /** Constructor used by Avro */ - private Pair(Schema schema) { - this.schema = schema; - } - private Pair(X first, Y second) { this.first = first; this.second = second; diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 6486bd7fd483..27cd96a39733 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -35,6 +35,7 @@ import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.io.Closer; @@ -77,7 +78,8 @@ public CloseableIterator iterator() { return iter; } - private static class ParallelIterator implements CloseableIterator { + @VisibleForTesting + static class ParallelIterator implements CloseableIterator { private final Iterator> tasks; private final Deque> yieldedTasks = new ArrayDeque<>(); private final ExecutorService workerPool; @@ -99,6 +101,7 @@ private ParallelIterator( } @Override + @SuppressWarnings("FutureReturnValueIgnored") public void close() { // close first, avoid new task submit this.closed.set(true); @@ -229,6 +232,11 @@ public synchronized T next() { } return queue.poll(); } + + @VisibleForTesting + int queueSize() { + return queue.size(); + } } private static class Task implements Supplier>>, Closeable { diff --git a/core/src/main/java/org/apache/iceberg/util/Tasks.java b/core/src/main/java/org/apache/iceberg/util/Tasks.java index 02d2b834311f..14804e040755 100644 --- a/core/src/main/java/org/apache/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -450,7 +450,9 @@ private void runTaskWithRetry(Task task, I item) thr } int delayMs = - (int) Math.min(minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), maxSleepTimeMs); + (int) + Math.min( + minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), (double) maxSleepTimeMs); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMs * 0.1))); LOG.warn("Retrying task after failure: {}", e.getMessage(), e); diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java index 18b452f98367..df96b90eb728 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -102,6 +102,7 @@ public ViewMetadata refresh() { } @Override + @SuppressWarnings("ImmutablesReferenceEquality") public void commit(ViewMetadata base, ViewMetadata metadata) { // if the metadata is already out of date, reject it if (base != current()) { diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 229650566ca8..836a1ddd80f5 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -46,7 +46,7 @@ public abstract class DeleteFileIndexTestBase< @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } static final DeleteFile FILE_A_POS_1 = diff --git a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java index a4e964b017ba..98d2d8f38af6 100644 --- a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java @@ -40,7 +40,7 @@ public abstract class MetadataTableScanTestBase extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } protected Set scannedPaths(TableScan scan) { diff --git a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java index 3ba74c3b4b71..13e96869b454 100644 --- a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java @@ -47,7 +47,7 @@ public abstract class ScanPlanningAndReportingTestBase< @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } protected abstract ScanT newScan(Table table); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 2322062dad85..e03a1efd5156 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -174,7 +174,7 @@ public class TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter protected int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestBatchScans.java b/core/src/test/java/org/apache/iceberg/TestBatchScans.java index c7210486e05e..1597f44f6338 100644 --- a/core/src/test/java/org/apache/iceberg/TestBatchScans.java +++ b/core/src/test/java/org/apache/iceberg/TestBatchScans.java @@ -34,7 +34,7 @@ public class TestBatchScans extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index bbba192fab7f..ecfaac618e2c 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -37,7 +37,7 @@ public class TestCommitReporting extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } @TestTemplate @@ -83,6 +83,9 @@ public void addAndDeleteDataFiles() { assertThat(metrics.removedFilesSizeInBytes().value()).isEqualTo(20L); assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(0L); + + assertThat(metrics.totalDataManifestFiles().value()).isEqualTo(1L); + assertThat(metrics.totalDeleteManifestFiles()).isNull(); } @TestTemplate @@ -122,6 +125,9 @@ public void addAndDeleteDeleteFiles() { assertThat(metrics.addedFilesSizeInBytes().value()).isEqualTo(30L); assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(30L); + assertThat(metrics.totalDataManifestFiles()).isNull(); + assertThat(metrics.totalDeleteManifestFiles().value()).isEqualTo(1L); + // now remove those 2 positional + 1 equality delete files table .newRewrite() @@ -153,6 +159,9 @@ public void addAndDeleteDeleteFiles() { assertThat(metrics.removedFilesSizeInBytes().value()).isEqualTo(30L); assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(0L); + + assertThat(metrics.totalDataManifestFiles()).isNull(); + assertThat(metrics.totalDeleteManifestFiles().value()).isEqualTo(1L); } @TestTemplate @@ -191,5 +200,7 @@ public void addAndDeleteManifests() throws IOException { assertThat(metrics.addedDataFiles().value()).isEqualTo(1L); assertThat(metrics.addedRecords().value()).isEqualTo(1L); assertThat(metrics.addedFilesSizeInBytes().value()).isEqualTo(10L); + assertThat(metrics.totalDataManifestFiles().value()).isEqualTo(2L); + assertThat(metrics.totalDeleteManifestFiles()).isNull(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 0c6b50b37792..766dd85a0655 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -40,7 +40,7 @@ public class TestCreateTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java index 60e79aad91ce..9bce4e60a4f3 100644 --- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java +++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java @@ -34,7 +34,7 @@ public class TestEntriesMetadataTable extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 9dd479ecf0fc..8125c528d9c3 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -39,7 +39,7 @@ public class TestFastAppend extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate @@ -324,6 +324,56 @@ public void testRecoveryWithoutManifestList() { assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); } + @TestTemplate + public void testWriteNewManifestsIdempotency() { + // inject 3 failures, the last try will succeed + TestTables.TestTableOperations ops = table.ops(); + ops.failCommits(3); + + AppendFiles append = table.newFastAppend().appendFile(FILE_B); + Snapshot pending = append.apply(); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); + assertThat(new File(newManifest.path())).exists(); + + append.commit(); + + TableMetadata metadata = readMetadata(); + + // contains only a single manifest, does not duplicate manifests on retries + validateSnapshot(null, metadata.currentSnapshot(), FILE_B); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); + assertThat(listManifestFiles(tableDir)).containsExactly(new File(newManifest.path())); + } + + @TestTemplate + public void testWriteNewManifestsCleanup() { + // append file, stage changes with apply() but do not commit + AppendFiles append = table.newFastAppend().appendFile(FILE_A); + Snapshot pending = append.apply(); + ManifestFile oldManifest = pending.allManifests(FILE_IO).get(0); + assertThat(new File(oldManifest.path())).exists(); + + // append file, stage changes with apply() but do not commit + // validate writeNewManifests deleted the old staged manifest + append.appendFile(FILE_B); + Snapshot newPending = append.apply(); + List manifestFiles = newPending.allManifests(FILE_IO); + assertThat(manifestFiles).hasSize(1); + ManifestFile newManifest = manifestFiles.get(0); + assertThat(newManifest.path()).isNotEqualTo(oldManifest.path()); + + append.commit(); + TableMetadata metadata = readMetadata(); + + // contains only a single manifest, old staged manifest is deleted + validateSnapshot(null, metadata.currentSnapshot(), FILE_A, FILE_B); + assertThat(new File(oldManifest.path())).doesNotExist(); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).containsExactly(newManifest); + assertThat(listManifestFiles(tableDir)).containsExactly(new File(newManifest.path())); + } + @TestTemplate public void testAppendManifestWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 137e7897385b..c4a9fdf2340a 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -31,14 +31,6 @@ public class TestFileScanTaskParser { @Test public void testNullArguments() { - assertThatThrownBy(() -> FileScanTaskParser.toJson(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file scan task: null"); - - assertThatThrownBy(() -> FileScanTaskParser.fromJson((String) null, true)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid JSON string for file scan task: null"); - assertThatThrownBy(() -> ScanTaskParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid scan task: null"); @@ -48,28 +40,6 @@ public void testNullArguments() { .hasMessage("Invalid JSON string for scan task: null"); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testFileScanTaskParser(boolean caseSensitive) { - PartitionSpec spec = TestBase.SPEC; - FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); - String jsonStr = FileScanTaskParser.toJson(fileScanTask); - assertThat(jsonStr).isEqualTo(fileScanTaskJsonWithoutTaskType()); - FileScanTask deserializedTask = FileScanTaskParser.fromJson(jsonStr, caseSensitive); - assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); - } - - /** Test backward compatibility where task-type field is absent from the JSON string */ - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testFileScanTaskParserWithoutTaskTypeField(boolean caseSensitive) { - PartitionSpec spec = TestBase.SPEC; - FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); - FileScanTask deserializedTask = - FileScanTaskParser.fromJson(fileScanTaskJsonWithoutTaskType(), caseSensitive); - assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); - } - @ParameterizedTest @ValueSource(booleans = {true, false}) public void testScanTaskParser(boolean caseSensitive) { diff --git a/core/src/test/java/org/apache/iceberg/TestFindFiles.java b/core/src/test/java/org/apache/iceberg/TestFindFiles.java index 68d757de5cdb..191dbd384197 100644 --- a/core/src/test/java/org/apache/iceberg/TestFindFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestFindFiles.java @@ -36,7 +36,7 @@ public class TestFindFiles extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java index 2f6a01c6e603..4a9f18581d09 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -71,7 +71,7 @@ public void testFormatVersionUpgradeNotSupported() { base, base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot upgrade table to unsupported format version: v3 (supported: v2)"); + .hasMessage("Cannot upgrade table to unsupported format version: v4 (supported: v3)"); assertThat(ops.current().formatVersion()).isEqualTo(1); } diff --git a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java index fe6b9b0c763c..ecd6a14ffefb 100644 --- a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java @@ -43,7 +43,7 @@ public class TestIncrementalDataTableScan extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java index 275b89df3695..8a4f5db256f0 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java @@ -28,7 +28,7 @@ public class TestLocalFilterFiles @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 93dd1a8555ce..7afb69483490 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -34,7 +34,7 @@ public class TestLocationProvider extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } // publicly visible for testing to be dynamically loaded diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java index b5f6d05cc6a0..37ccbb8bb845 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java @@ -30,7 +30,7 @@ public class TestManifestCleanup extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java index 4c60a943f76c..175178e48167 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java @@ -39,7 +39,7 @@ public class TestManifestReaderStats extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final Map VALUE_COUNT = ImmutableMap.of(3, 3L); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index eb7910a79fc5..7dd3ea2d1ce7 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -38,7 +38,7 @@ public class TestManifestWriter extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 250; diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index 8125e064f0d0..9e535dd77747 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -40,7 +40,7 @@ public class TestMetadataTableFilters extends TestBase { private static final Set AGG_FILE_TABLES = Sets.newHashSet( MetadataTableType.ALL_DATA_FILES, - MetadataTableType.ALL_DATA_FILES, + MetadataTableType.ALL_DELETE_FILES, MetadataTableType.ALL_FILES, MetadataTableType.ALL_ENTRIES); @@ -132,9 +132,9 @@ private int expectedScanTaskCount(int partitions) { } case DATA_FILES: case DELETE_FILES: - case ALL_DELETE_FILES: return partitions; case ALL_DATA_FILES: + case ALL_DELETE_FILES: return partitions * 2; // ScanTask for Data Manifest in DELETED and ADDED states case ALL_FILES: case ALL_ENTRIES: diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 0a14a89c01b5..755eb57de8e8 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -55,7 +55,7 @@ public class TestMetadataTableScans extends MetadataTableScanTestBase { private void preparePartitionedTable(boolean transactional) { preparePartitionedTableData(transactional); - if (formatVersion == 2) { + if (formatVersion >= 2) { if (transactional) { table .newRowDelta() @@ -485,7 +485,7 @@ public void testPartitionsTableScanNoFilter() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanNoFilter); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -510,7 +510,7 @@ public void testPartitionsTableScanWithProjection() { assertThat(scanWithProjection.schema().asStruct()).isEqualTo(expected); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanWithProjection); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -552,7 +552,7 @@ public void testPartitionsTableScanAndFilter() { TableScan scanAndEq = partitionsTable.newScan().filter(andEquals); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanAndEq); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(2); } else { assertThat(entries).hasSize(1); @@ -574,7 +574,7 @@ public void testPartitionsTableScanLtFilter() { TableScan scanLtAnd = partitionsTable.newScan().filter(ltAnd); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanLtAnd); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -598,7 +598,7 @@ public void testPartitionsTableScanOrFilter() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanOr); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -619,7 +619,7 @@ public void testPartitionsScanNotFilter() { TableScan scanNot = partitionsTable.newScan().filter(not); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanNot); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -639,7 +639,7 @@ public void testPartitionsTableScanInFilter() { TableScan scanSet = partitionsTable.newScan().filter(set); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanSet); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -659,7 +659,7 @@ public void testPartitionsTableScanNotNullFilter() { TableScan scanUnary = partitionsTable.newScan().filter(unary); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanUnary); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -719,8 +719,7 @@ public void testFilesTableScanWithDroppedPartition() throws IOException { @TestTemplate public void testDeleteFilesTableSelection() throws IOException { - assumeThat(formatVersion).as("Only V2 Tables Support Deletes").isGreaterThanOrEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.newFastAppend().appendFile(FILE_A).commit(); table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); @@ -960,7 +959,7 @@ public void testPartitionSpecEvolutionAdditive() { TableScan scan = metadataTable.newScan().filter(filter); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // Four data files and delete files of old spec, one new data file of new spec assertThat(entries).hasSize(9); } else { @@ -975,7 +974,7 @@ public void testPartitionSpecEvolutionAdditive() { scan = metadataTable.newScan().filter(filter); entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // 1 original data file and delete file written by old spec, plus 1 new data file written by // new spec assertThat(entries).hasSize(3); @@ -1026,7 +1025,7 @@ public void testPartitionSpecEvolutionRemoval() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // Four data and delete files of original spec, one data file written by new spec assertThat(entries).hasSize(9); } else { @@ -1187,7 +1186,7 @@ public void testPartitionsTableScanWithPlanExecutor() { })); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -1366,7 +1365,7 @@ public void testAllManifestsTableSnapshotNot() { @TestTemplate public void testPositionDeletesWithFilter() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1429,7 +1428,7 @@ public void testPositionDeletesBaseTableFilterEntriesLevel() { } private void testPositionDeletesBaseTableFilter(boolean transactional) { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(transactional); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1490,9 +1489,9 @@ private void testPositionDeletesBaseTableFilter(boolean transactional) { @TestTemplate public void testPositionDeletesWithBaseTableFilterNot() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - - // use identity rather than bucket partition spec, + assumeThat(formatVersion) + .as("Position deletes are not supported by V1 Tables") + .isNotEqualTo(1); // use identity rather than bucket partition spec, // as bucket.project does not support projecting notEq table.updateSpec().removeField("data_bucket").addField("id").commit(); PartitionSpec spec = table.spec(); @@ -1574,7 +1573,7 @@ public void testPositionDeletesWithBaseTableFilterNot() { @TestTemplate public void testPositionDeletesResiduals() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1603,7 +1602,7 @@ public void testPositionDeletesResiduals() { @TestTemplate public void testPositionDeletesUnpartitioned() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.updateSpec().removeField(Expressions.bucket("data", BUCKETS_NUMBER)).commit(); assertThat(table.spec().fields()).as("Table should now be unpartitioned").hasSize(0); @@ -1694,8 +1693,7 @@ public void testPositionDeletesUnpartitioned() { @TestTemplate public void testPositionDeletesManyColumns() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); UpdateSchema updateSchema = table.updateSchema(); for (int i = 0; i <= 2000; i++) { updateSchema.addColumn(String.valueOf(i), Types.IntegerType.get()); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index a2e5386d29df..ac96642319a3 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -163,8 +163,7 @@ public void testPartitionsTableScanWithAddPartitionOnNestedField() { @TestTemplate public void testPositionDeletesPartitionSpecRemoval() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.updateSpec().removeField("id").commit(); DeleteFile deleteFile = newDeleteFile(table.ops().current().spec().specId(), "nested.id=1"); diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index b95b92979f91..2c4849135f64 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -69,7 +69,7 @@ public abstract class TestMetrics { @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TempDir public Path temp; diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java index 31ae459df506..00e961097c34 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java @@ -47,7 +47,7 @@ public class TestMetricsModes { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TempDir private Path temp; diff --git a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java index 733bb0bb38fd..8c6f18619ac0 100644 --- a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java +++ b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java @@ -34,7 +34,7 @@ public class TestMicroBatchBuilder extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index ff54929504ee..a324b8af2e70 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -47,7 +47,7 @@ public class TestPartitionSpecInfo { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index 3a6d2017eb82..b1b481dd5305 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -50,7 +50,7 @@ public class TestReplaceTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 176f61079fc3..b7233659c509 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; +import static org.apache.iceberg.TestSnapshot.testManifestStats; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; @@ -48,7 +49,7 @@ public class TestRewriteManifests extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate @@ -62,6 +63,8 @@ public void testRewriteManifestsAppendedDirectly() throws IOException { "manifest-file-1.avro", manifestEntry(ManifestEntry.Status.ADDED, null, FILE_A)); table.newFastAppend().appendManifest(newManifest).commit(); + + testManifestStats(table); long appendId = table.currentSnapshot().snapshotId(); assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); @@ -71,6 +74,7 @@ public void testRewriteManifestsAppendedDirectly() throws IOException { List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(1); + testManifestStats(table); validateManifestEntries( manifests.get(0), ids(appendId), files(FILE_A), statuses(ManifestEntry.Status.EXISTING)); } @@ -87,6 +91,8 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { table.newFastAppend().appendManifest(newManifest).commit(); + testManifestStats(table); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); AtomicInteger scanThreadsIndex = new AtomicInteger(0); table @@ -106,6 +112,7 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(1); + testManifestStats(table); assertThat(scanThreadsIndex.get()) .as("Thread should be created in provided pool") .isGreaterThan(0); @@ -129,11 +136,13 @@ public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOExceptio assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); + testManifestStats(table); table.rewriteManifests().clusterBy(file -> "").commit(); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(1); + testManifestStats(table); // get the correct file order List files; List ids; @@ -162,12 +171,14 @@ public void testReplaceManifestsSeparate() { assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); - // cluster by path will split the manifest into two + testManifestStats(table); + // cluster by path will split the manifest into two table.rewriteManifests().clusterBy(file -> file.path()).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(2); + testManifestStats(table); manifests.sort(Comparator.comparing(ManifestFile::path)); validateManifestEntries( @@ -185,6 +196,7 @@ public void testReplaceManifestsConsolidate() throws IOException { table.newFastAppend().appendFile(FILE_B).commit(); long appendIdB = table.currentSnapshot().snapshotId(); + testManifestStats(table); assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); // cluster by constant will combine manifests into one @@ -194,6 +206,7 @@ public void testReplaceManifestsConsolidate() throws IOException { List manifests = table.currentSnapshot().allManifests(table.io()); assertThat(manifests).hasSize(1); + testManifestStats(table); // get the file order correct List files; List ids; @@ -277,6 +290,8 @@ public void testReplaceManifestsMaxSize() { assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); + testManifestStats(table); + // cluster by constant will combine manifests into one but small target size will create one per // entry BaseRewriteManifests rewriteManifests = spy((BaseRewriteManifests) table.rewriteManifests()); @@ -319,6 +334,7 @@ public void testConcurrentRewriteManifest() throws IOException { }) .commit(); + testManifestStats(table); assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); // commit the rewrite manifests in progress - this should perform a full rewrite as the manifest @@ -362,6 +378,7 @@ public void testAppendDuringRewriteManifest() { table.newFastAppend().appendFile(FILE_B).commit(); long appendIdB = table.currentSnapshot().snapshotId(); + testManifestStats(table); assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); // commit the rewrite manifests in progress @@ -395,6 +412,7 @@ public void testRewriteManifestDuringAppend() { assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); + testManifestStats(table); // commit the append in progress append.commit(); long appendIdB = table.currentSnapshot().snapshotId(); @@ -422,6 +440,9 @@ public void testBasicManifestReplacement() throws IOException { ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); + + testManifestStats(table); + Snapshot secondSnapshot = table.currentSnapshot(); ManifestFile firstNewManifest = @@ -439,6 +460,8 @@ public void testBasicManifestReplacement() throws IOException { rewriteManifests.addManifest(secondNewManifest); rewriteManifests.commit(); + testManifestStats(table); + Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); assertThat(manifests).hasSize(3); @@ -480,12 +503,17 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + testManifestStats(table); + Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); + + testManifestStats(table); + Snapshot secondSnapshot = table.currentSnapshot(); ManifestFile firstNewManifest = @@ -503,6 +531,8 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc rewriteManifests.addManifest(secondNewManifest); rewriteManifests.commit(); + testManifestStats(table); + Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); assertThat(manifests).hasSize(3); @@ -573,6 +603,8 @@ public void testWithMultiplePartitionSpec() throws IOException { table.newAppend().appendFile(newFileZ).commit(); + testManifestStats(table); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(3); RewriteManifests rewriteManifests = table.rewriteManifests(); diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java index b326274842a3..10c22b759988 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -39,7 +39,7 @@ public class TestScanSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java index 0d5b1bc7066b..3fc691ce942a 100644 --- a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java @@ -55,7 +55,7 @@ public class TestScansAndSchemaEvolution { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java index 1bd1761ffc60..111693792f78 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java @@ -43,7 +43,7 @@ public class TestSchemaAndMappingUpdate extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaID.java b/core/src/test/java/org/apache/iceberg/TestSchemaID.java index 33ac84d20801..e3c58af1cbe3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaID.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaID.java @@ -36,7 +36,7 @@ public class TestSchemaID extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java index 3ebe047e90b7..9504f78f5345 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java @@ -29,7 +29,7 @@ public class TestSetPartitionStatistics extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java index 41941e3c6630..03c644117a95 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java @@ -31,7 +31,7 @@ public class TestSetStatistics extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index 2ec6abd4e428..be02568d089c 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg; +import static org.apache.iceberg.SnapshotSummary.TOTAL_DATA_MANIFEST_FILES; +import static org.apache.iceberg.SnapshotSummary.TOTAL_DELETE_MANIFEST_FILES; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; @@ -33,18 +35,22 @@ public class TestSnapshot extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate public void testAppendFilesFromTable() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + testManifestStats(table); + // collect data files from deserialization Iterable filesToAdd = table.currentSnapshot().addedDataFiles(table.io()); table.newDelete().deleteFile(FILE_A).deleteFile(FILE_B).commit(); + testManifestStats(table); + Snapshot oldSnapshot = table.currentSnapshot(); AppendFiles fastAppend = table.newFastAppend(); @@ -53,6 +59,9 @@ public void testAppendFilesFromTable() { } Snapshot newSnapshot = fastAppend.apply(); + + testManifestStats(table); + validateSnapshot(oldSnapshot, newSnapshot, FILE_A, FILE_B); } @@ -60,6 +69,8 @@ public void testAppendFilesFromTable() { public void testAppendFoundFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + testManifestStats(table); + Iterable filesToAdd = FindFiles.in(table) .inPartition(table.spec(), StaticDataTask.Row.of(0)) @@ -68,6 +79,8 @@ public void testAppendFoundFiles() { table.newDelete().deleteFile(FILE_A).deleteFile(FILE_B).commit(); + testManifestStats(table); + Snapshot oldSnapshot = table.currentSnapshot(); AppendFiles fastAppend = table.newFastAppend(); @@ -76,6 +89,8 @@ public void testAppendFoundFiles() { } Snapshot newSnapshot = fastAppend.apply(); + + testManifestStats(table); validateSnapshot(oldSnapshot, newSnapshot, FILE_A, FILE_B); } @@ -83,16 +98,24 @@ public void testAppendFoundFiles() { public void testCachedDataFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + testManifestStats(table); + table.updateSpec().addField(Expressions.truncate("data", 2)).commit(); + testManifestStats(table); + DataFile secondSnapshotDataFile = newDataFile("data_bucket=8/data_trunc_2=aa"); table.newFastAppend().appendFile(secondSnapshotDataFile).commit(); + testManifestStats(table); + DataFile thirdSnapshotDataFile = newDataFile("data_bucket=8/data_trunc_2=bb"); table.newOverwrite().deleteFile(FILE_A).addFile(thirdSnapshotDataFile).commit(); + testManifestStats(table); + Snapshot thirdSnapshot = table.currentSnapshot(); Iterable removedDataFiles = thirdSnapshot.removedDataFiles(FILE_IO); @@ -118,8 +141,12 @@ public void testCachedDeleteFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + testManifestStats(table); + table.updateSpec().addField(Expressions.truncate("data", 2)).commit(); + testManifestStats(table); + int specId = table.spec().specId(); DataFile secondSnapshotDataFile = newDataFile("data_bucket=8/data_trunc_2=aa"); @@ -131,6 +158,8 @@ public void testCachedDeleteFiles() { .addDeletes(secondSnapshotDeleteFile) .commit(); + testManifestStats(table); + DeleteFile thirdSnapshotDeleteFile = newDeleteFile(specId, "data_bucket=8/data_trunc_2=aa"); ImmutableSet replacedDeleteFiles = ImmutableSet.of(secondSnapshotDeleteFile); @@ -141,6 +170,8 @@ public void testCachedDeleteFiles() { .rewriteFiles(ImmutableSet.of(), replacedDeleteFiles, ImmutableSet.of(), newDeleteFiles) .commit(); + testManifestStats(table); + Snapshot thirdSnapshot = table.currentSnapshot(); Iterable removedDeleteFiles = thirdSnapshot.removedDeleteFiles(FILE_IO); @@ -275,4 +306,13 @@ private void runAddedDeleteFileSequenceNumberTest( .as("File sequence number mismatch") .isEqualTo(expectedSequenceNumber); } + + public static void testManifestStats(Table table) { + assertThat(table.currentSnapshot().summary().get(TOTAL_DATA_MANIFEST_FILES)) + .isEqualTo(String.valueOf(table.currentSnapshot().dataManifests(table.io()).size())); + + int deletedManifestCount = table.currentSnapshot().deleteManifests(table.io()).size(); + assertThat(table.currentSnapshot().summary().get(TOTAL_DELETE_MANIFEST_FILES)) + .isEqualTo(deletedManifestCount == 0 ? null : String.valueOf(deletedManifestCount)); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java index 89312201265d..96dde0f48eb7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java @@ -43,7 +43,7 @@ public class TestSnapshotLoading extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private Snapshot currentSnapshot; diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java index 88233dd99097..61dd082701c2 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java @@ -51,7 +51,7 @@ public class TestSnapshotManager extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java index 7ce59e9df1c9..6d339c0cbeaf 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java @@ -33,7 +33,7 @@ public class TestSnapshotSelection extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 053a9c374178..d4db286a82f1 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -32,7 +32,7 @@ public class TestSnapshotSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate @@ -89,6 +89,29 @@ public void testFileSizeSummaryWithDeletes() { .containsEntry(SnapshotSummary.ADD_POS_DELETE_FILES_PROP, "1"); } + @TestTemplate + public void testManifestStatSummaryWithDeletes() { + if (formatVersion == 1) { + return; + } + + // fast append + table.newFastAppend().appendFile(FILE_A).commit(); + Map summary = table.currentSnapshot().summary(); + + assertThat(summary) + .containsEntry(SnapshotSummary.TOTAL_DATA_MANIFEST_FILES, "1") + .doesNotContainKey(SnapshotSummary.TOTAL_DELETE_MANIFEST_FILES); + + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + table.refresh(); + summary = table.currentSnapshot().summary(); + + assertThat(summary) + .containsEntry(SnapshotSummary.TOTAL_DATA_MANIFEST_FILES, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_MANIFEST_FILES, "1"); + } + @TestTemplate public void testIcebergVersionInSummary() { table.newFastAppend().appendFile(FILE_A).commit(); diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index a3ba69a808b3..ad773192b417 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -76,7 +76,7 @@ public class TestSortOrder { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index 04bb2ae215d8..ea2142982382 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -58,7 +58,7 @@ public class TestSplitPlanning extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java index f47968434bfe..94a3d35b35a6 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java @@ -35,7 +35,7 @@ public class TestTableMetadataSerialization extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java index a4e587068e74..f327ef752947 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java @@ -36,7 +36,7 @@ public class TestTableUpdatePartitionSpec extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach @@ -287,4 +287,23 @@ public void testAddAfterLastFieldRemoved() { assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); assertThat(table.ops().current().lastAssignedPartitionId()).isEqualTo(1001); } + + @TestTemplate + public void testCommitUpdatedSpecWithoutSettingNewDefault() { + PartitionSpec originalSpec = table.spec(); + table.updateSpec().addField("id").addNonDefaultSpec().commit(); + + assertThat(table.spec()) + .as("Should not set the default spec for the table") + .isSameAs(originalSpec); + + assertThat(table.specs().get(1)) + .as("The new spec created for the table") + .isEqualTo( + PartitionSpec.builderFor(table.schema()) + .withSpecId(1) + .bucket("data", 16) + .identity("id") + .build()); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java index 08714dec01f1..656dd782cfe4 100644 --- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java @@ -35,7 +35,7 @@ public class TestTimestampPartitions extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 393494da1283..8fed7134fae1 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -42,7 +42,7 @@ public class TestTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java index 33b003cfd8c5..97f695315e16 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java @@ -54,7 +54,7 @@ public class TestUpdatePartitionSpec extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java index 58b83f43f399..b1e8e57850e8 100644 --- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java +++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java @@ -35,7 +35,7 @@ public class TestWapWorkflow extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java index a9a23d587ac9..77d16d3bc821 100644 --- a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java +++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java @@ -41,7 +41,7 @@ public class TestSizeBasedRewriter extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java index ceffeb3749da..bb98925f504c 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java +++ b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java @@ -40,7 +40,7 @@ public class TestOutputFileFactory extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final int PARTITION_ID = 1; diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java index b9ae9dc273f2..2161b40a60b5 100644 --- a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java +++ b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java @@ -36,7 +36,7 @@ public class TestMappingUpdates extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 53d57bee510f..2c928c06e52b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -288,11 +288,7 @@ public T handleRequest( if (asNamespaceCatalog != null) { Namespace ns; if (vars.containsKey("parent")) { - ns = - Namespace.of( - RESTUtil.NAMESPACE_SPLITTER - .splitToStream(vars.get("parent")) - .toArray(String[]::new)); + ns = RESTUtil.decodeNamespace(vars.get("parent")); } else { ns = Namespace.empty(); } diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java index 954f5130eca2..f456bb4d354d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java @@ -20,6 +20,9 @@ import static java.lang.String.format; +import jakarta.servlet.http.HttpServlet; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.servlet.http.HttpServletResponse; import java.io.IOException; import java.io.InputStreamReader; import java.io.Reader; @@ -30,9 +33,6 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; -import javax.servlet.http.HttpServlet; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.HttpHeaders; import org.apache.iceberg.exceptions.RESTException; diff --git a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java index 4910732f6e35..c259bbd0a7e2 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -21,12 +21,9 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; -import java.lang.reflect.Field; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -40,6 +37,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Multiset; +import org.apache.iceberg.util.ParallelIterable.ParallelIterator; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; @@ -64,20 +62,17 @@ public CloseableIterator iterator() { }); ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); - CloseableIterator iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); Awaitility.await("Queue is populated") .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> queueHasElements(iterator, queue)); + .untilAsserted(() -> queueHasElements(iterator)); iterator.close(); Awaitility.await("Queue is cleared") .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertThat(queue).isEmpty()); + .untilAsserted(() -> assertThat(iterator.queueSize()).isEqualTo(0)); } @Test @@ -124,20 +119,21 @@ public CloseableIterator iterator() { }); ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); - CloseableIterator iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); Awaitility.await("Queue is populated") .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> queueHasElements(iterator, queue)); + .untilAsserted(() -> queueHasElements(iterator)); iterator.close(); Awaitility.await("Queue is cleared") .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertThat(queue).as("Queue is not empty after cleaning").isEmpty()); + .untilAsserted( + () -> + assertThat(iterator.queueSize()) + .as("Queue is not empty after cleaning") + .isEqualTo(0)); } @Test @@ -159,17 +155,14 @@ public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchF ExecutorService executor = Executors.newCachedThreadPool(); ParallelIterable parallelIterable = new ParallelIterable<>(iterables, executor, maxQueueSize); - CloseableIterator iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); Multiset actualValues = HashMultiset.create(); while (iterator.hasNext()) { - assertThat(queue) - .as("iterator internal queue") - .hasSizeLessThanOrEqualTo(maxQueueSize + iterables.size()); + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(maxQueueSize + iterables.size()); actualValues.add(iterator.next()); } @@ -181,9 +174,9 @@ public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchF executor.shutdownNow(); } - private void queueHasElements(CloseableIterator iterator, Queue queue) { + private void queueHasElements(ParallelIterator iterator) { assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); - assertThat(queue).isNotEmpty(); + assertThat(iterator.queueSize()).as("queue size").isGreaterThan(0); } } diff --git a/core/src/test/resources/TableMetadataUnsupportedVersion.json b/core/src/test/resources/TableMetadataUnsupportedVersion.json index 0633a71d24d5..c40a0c9cd5ae 100644 --- a/core/src/test/resources/TableMetadataUnsupportedVersion.json +++ b/core/src/test/resources/TableMetadataUnsupportedVersion.json @@ -1,5 +1,5 @@ { - "format-version": 3, + "format-version": 4, "table-uuid": "d20125c8-7284-442c-9aea-15fee620737c", "location": "s3://bucket/test/location", "last-updated-ms": 1602638573874, @@ -33,4 +33,4 @@ "properties": {}, "current-snapshot-id": -1, "snapshots": [] -} \ No newline at end of file +} diff --git a/deploy.gradle b/deploy.gradle index 3e2eda2a5a60..462448303253 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (project.hasProperty('release') && jdkVersion != '8') { - throw new GradleException("Releases must be built with Java 8") +if (project.hasProperty('release') && jdkVersion != '11') { + throw new GradleException("Releases must be built with Java 11") } subprojects { diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index 05bf3c4253f2..29cf31e5f423 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -19,7 +19,7 @@ # SCALA_VERSION=2.12 -FLINK_VERSIONS=1.17,1.18,1.19 +FLINK_VERSIONS=1.18,1.19,1.20 SPARK_VERSIONS=3.3,3.4,3.5 HIVE_VERSIONS=2,3 diff --git a/docs/docs/aws.md b/docs/docs/aws.md index bba968fa5586..43e540c20673 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -468,8 +468,8 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata --conf spark.sql.catalog.my_catalog.type=glue \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.use-arn-region-enabled=false \ - --conf spark.sql.catalog.test.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ - --conf spark.sql.catalog.test.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap ``` For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap` access-point for all S3 operations. diff --git a/docs/docs/branching.md b/docs/docs/branching.md index 3379264d8a5f..f00defda665b 100644 --- a/docs/docs/branching.md +++ b/docs/docs/branching.md @@ -148,7 +148,7 @@ SELECT * FROM db.table.branch_test_branch; Modify the table's schema by dropping the `col` column and adding a new column named `new_col`: ```sql -ALTER TABLE db.table drop column float; +ALTER TABLE db.table drop column col; ALTER TABLE db.table add column new_col date; diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index b606d849a692..494ca698533a 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -288,6 +288,7 @@ order by made_current_at; | 2019-02-09 16:24:30.13 | delete | 29641004024753 | false | application_1520379288616_151109 | | 2019-02-09 16:32:47.336 | append | 57897183625154 | true | application_1520379288616_155055 | | 2019-02-08 03:47:55.948 | overwrite | 51792995261850 | true | application_1520379288616_152431 | + ### Entries To show all the table's current manifest entries for both data and delete files. diff --git a/flink/build.gradle b/flink/build.gradle index f049ff69b059..17ed630cc235 100644 --- a/flink/build.gradle +++ b/flink/build.gradle @@ -19,11 +19,6 @@ def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") - -if (flinkVersions.contains("1.17")) { - apply from: file("$projectDir/v1.17/build.gradle") -} - if (flinkVersions.contains("1.18")) { apply from: file("$projectDir/v1.18/build.gradle") } @@ -31,3 +26,7 @@ if (flinkVersions.contains("1.18")) { if (flinkVersions.contains("1.19")) { apply from: file("$projectDir/v1.19/build.gradle") } + +if (flinkVersions.contains("1.20")) { + apply from: file("$projectDir/v1.20/build.gradle") +} \ No newline at end of file diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 45af9241b743..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index 3a8f5ccc6c03..4bd85bbd97b4 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -102,22 +102,4 @@ public static RowData clone( return ret; } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index c0e228965ddd..e4cba174f0f2 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 3b21fbae315a..4bfde7204acf 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java index a164d83ac3b0..781bcc646023 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -98,7 +98,11 @@ long[] subtaskWeightsExcludingCloseCost() { return subtaskWeightsExcludingCloseCost; } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.size() == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java index 0abb030c2279..9d8167460a1b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -93,6 +93,8 @@ Map keyAssignments() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..ccbd0d9997ed 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 6c9a855bc149..280a126a46ce 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -37,7 +38,8 @@ import org.slf4j.LoggerFactory; abstract class AbstractIcebergEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -95,6 +97,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 399d7aaff64c..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = - new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..a74226092f38 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -43,7 +43,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..b73300e3f170 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..d52d54e159e6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 013b98e3b82b..b709c0058f7d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
    + *
  • age <= 15 + *
  • age > 15 && age <= 32 + *
  • age >32 && age <= 60 + *
  • age > 60 + *
+ */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 749cbf89338a..e0e2bf5e61e2 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -67,7 +67,7 @@ public class TestIcebergSourceContinuous { @TempDir protected Path temporaryFolder; @RegisterExtension - public static MiniClusterExtension miniClusterExtension = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); @RegisterExtension diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..95d0b90b6ca9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + sql("DROP DATABASE %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d6cf6791270e..57ee7baf202c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index c3917165753d..007b423e592a 100644 --- a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -27,6 +28,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -66,6 +69,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -82,10 +87,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index 3a8f5ccc6c03..4bd85bbd97b4 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -102,22 +102,4 @@ public static RowData clone( return ret; } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index c0e228965ddd..e4cba174f0f2 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 3b21fbae315a..4bfde7204acf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java index a164d83ac3b0..781bcc646023 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -98,7 +98,11 @@ long[] subtaskWeightsExcludingCloseCost() { return subtaskWeightsExcludingCloseCost; } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.size() == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java index 0abb030c2279..9d8167460a1b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -93,6 +93,8 @@ Map keyAssignments() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java index 91ed3c4adea3..062ff68d5d85 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -118,26 +118,4 @@ protected String getFullQualifiedTableName(String tableName) { static String getURI(HiveConf conf) { return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 399d7aaff64c..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = - new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } + + protected static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 773d22e19e64..401960c3591b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -27,7 +27,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -41,10 +40,10 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -public abstract class TestBase extends TestBaseUtils { +public abstract class TestBase extends SqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; @@ -72,6 +71,7 @@ public static void stopMetastore() throws Exception { TestBase.catalog = null; } + @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { synchronized (this) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index 3f66174049a4..2978a92945a2 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -18,36 +18,21 @@ */ package org.apache.iceberg.flink; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -91,7 +76,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); @@ -169,39 +154,6 @@ public void testOverwriteTable() throws Exception { icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); } - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); - String insertSQL = - String.format( - "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, SOURCE_TABLE); - ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); - Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); - Transformation committer = dummySink.getInputs().get(0); - Transformation writer = committer.getInputs().get(0); - - assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); - writer - .getInputs() - .forEach( - input -> - assertThat(input.getParallelism()) - .as("Should have the expected parallelism.") - .isEqualTo(isStreamingJob ? 2 : 4)); - } - @TestTemplate public void testReplacePartitions() throws Exception { assumeThat(isStreamingJob) @@ -289,70 +241,4 @@ public void testInsertIntoPartition() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); } } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java new file mode 100644 index 000000000000..482cfd110bde --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List parameters() { + return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + } + + protected synchronized TableEnvironment getTableEnv() { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + return tEnv; + } + + @BeforeEach + public void before() { + String warehouseLocation = "file:" + warehouseRoot.getPath(); + this.catalog = new HadoopCatalog(new Configuration(), warehouseLocation); + Map config = Maps.newHashMap(); + config.put("type", "iceberg"); + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HADOOP); + config.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + sql("CREATE CATALOG %s WITH %s", CATALOG, toWithClause(config)); + + sql("CREATE DATABASE %s", FLINK_DATABASE); + sql("USE CATALOG %s", CATALOG); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE, FileFormat.PARQUET.name()); + } + + @AfterEach + public void clean() throws Exception { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, TABLE); + dropDatabase(FLINK_DATABASE, true); + BoundedTableFactory.clearDataSets(); + + dropCatalog(CATALOG, true); + catalog.close(); + } + + @TestTemplate + public void testWriteParallelism() { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); + String insertSQL = + String.format( + "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", + TABLE, SOURCE_TABLE); + ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); + Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); + Transformation committer = dummySink.getInputs().get(0); + Transformation writer = committer.getInputs().get(0); + + assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); + writer + .getInputs() + .forEach( + input -> + assertThat(input.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4)); + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index baf13017ff99..c5becb6caca1 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index c978ec6f8bd4..47f5485df879 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); @@ -319,10 +319,6 @@ private String databaseName() { return properties.getOrDefault("catalog-database", "default_database"); } - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - private String createWarehouse() { try { return String.format( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 269ae681b02e..272e0b693fd3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -34,7 +34,7 @@ class OperatorTestBase { static final String TABLE_NAME = "test_table"; @RegisterExtension - protected static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..b778037c559c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,20 +18,11 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - import java.io.IOException; -import java.util.Collections; import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -39,38 +30,19 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @RegisterExtension - public static MiniClusterExtension miniClusterResource = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - @Parameter(index = 0) private FileFormat format; @@ -100,7 +72,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { - table = + this.table = CATALOG_EXTENSION .catalog() .createTable( @@ -111,14 +83,14 @@ public void before() throws IOException { : PartitionSpec.unpartitioned(), ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = + this.env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = CATALOG_EXTENSION.tableLoader(); + this.tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -141,246 +113,13 @@ public void testWriteRowData() throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); } - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - @TestTemplate public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); + testWriteRow(parallelism, null, DistributionMode.NONE); } @TestTemplate public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @TestTemplate - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); - } - } - } - - @TestTemplate - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @TestTemplate - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); - rightTable.refresh(); - assertThat(rightTable.currentSnapshot().summary()) - .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) - .containsEntry("direction", "rightTable"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @TestTemplate - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index b38aa6b50ce6..9ce36cc1e8d0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -18,29 +18,52 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkIcebergSinkBase { - protected Table table; - protected StreamExecutionEnvironment env; + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + protected static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); protected static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List createRows(String prefix) { protected List convertToRowData(List rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java new file mode 100644 index 000000000000..75e397d3f203 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * This tests the distribution mode of Flink sink. Extract them separately since it is unnecessary + * to test different file formats (Avro, Orc, Parquet) like in {@link TestFlinkIcebergSink}. + * Removing the file format dimension reduces the number of combinations from 12 to 4, which helps + * reduce test run time. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private final FileFormat format = FileFormat.PARQUET; + + @Parameter(index = 0) + private int parallelism; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "parallelism = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {1, true}, + {1, false}, + {2, true}, + {2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).isGreaterThan(3); + } + } + } + + @TestTemplate + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testPartitionWriteMode() throws Exception { + testWriteRow(parallelism, null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java similarity index 53% rename from flink/v1.17/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/TestFlinkIcebergSinkExtended.java index 8cad35c859c4..36a59b20431c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -29,22 +28,15 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -54,53 +46,21 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.Test; -@ExtendWith(ParameterizedTestExtension.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @RegisterExtension - public static MiniClusterExtension miniClusterResource = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - - @Parameter(index = 0) - private FileFormat format; - - @Parameter(index = 1) - private int parallelism; - - @Parameter(index = 2) - private boolean partitioned; - - @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") - public static Object[][] parameters() { - return new Object[][] { - {FileFormat.AVRO, 1, true}, - {FileFormat.AVRO, 1, false}, - {FileFormat.AVRO, 2, true}, - {FileFormat.AVRO, 2, false}, - {FileFormat.ORC, 1, true}, - {FileFormat.ORC, 1, false}, - {FileFormat.ORC, 2, true}, - {FileFormat.ORC, 2, false}, - {FileFormat.PARQUET, 1, true}, - {FileFormat.PARQUET, 1, false}, - {FileFormat.PARQUET, 2, true}, - {FileFormat.PARQUET, 2, false} - }; - } +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; @BeforeEach public void before() throws IOException { - table = + this.table = CATALOG_EXTENSION .catalog() .createTable( @@ -111,135 +71,17 @@ public void before() throws IOException { : PartitionSpec.unpartitioned(), ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = + this.env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = CATALOG_EXTENSION.tableLoader(); - } - - @TestTemplate - public void testWriteRowData() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - - @TestTemplate - public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); - } - - @TestTemplate - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @TestTemplate - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); - } - } - } - - @TestTemplate - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @TestTemplate - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } + this.tableLoader = CATALOG_EXTENSION.tableLoader(); } - @TestTemplate - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate + @Test public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); @@ -271,7 +113,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -320,27 +162,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { .containsEntry("direction", "rightTable"); } - @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @TestTemplate + @Test public void testOverrideWriteConfigWithUnknownFileFormat() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); @@ -360,7 +182,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .hasMessage("Invalid file format: UNRECOGNIZED"); } - @TestTemplate + @Test public void testWriteRowWithTableRefreshInterval() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
    + *
  • age <= 15 + *
  • age > 15 && age <= 32 + *
  • age >32 && age <= 60 + *
  • age > 60 + *
+ */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 3285a16a1277..41b023b93617 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -55,7 +55,7 @@ public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_SLOTS = 3; @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUM_TASK_MANAGERS) @@ -144,7 +144,7 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the even subtask indices with the first attempt to sleep to trigger speculative + // Put the subtasks with the first attempt to sleep to trigger speculative // execution if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index fcf5c1479df5..97ed4ca1e93f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = diff --git a/flink/v1.17/build.gradle b/flink/v1.20/build.gradle similarity index 91% rename from flink/v1.17/build.gradle rename to flink/v1.20/build.gradle index 0278e4dc3b73..f2e1fb51a1f4 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.20/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.17' +String flinkMajorVersion = '1.20' String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { @@ -32,15 +32,15 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') - compileOnly libs.flink117.avro + compileOnly libs.flink120.avro // for dropwizard histogram metrics implementation - compileOnly libs.flink117.metrics.dropwizard - compileOnly libs.flink117.streaming.java - compileOnly "${libs.flink117.streaming.java.get().module}:${libs.flink117.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink117.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" - compileOnly libs.flink117.connector.base - compileOnly libs.flink117.connector.files + compileOnly libs.flink120.metrics.dropwizard + compileOnly libs.flink120.streaming.java + compileOnly "${libs.flink120.streaming.java.get().module}:${libs.flink120.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink120.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink120.get()}" + compileOnly libs.flink120.connector.base + compileOnly libs.flink120.connector.files compileOnly libs.hadoop2.hdfs compileOnly libs.hadoop2.common @@ -68,13 +68,13 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation libs.datasketches - testImplementation libs.flink117.connector.test.utils - testImplementation libs.flink117.core - testImplementation libs.flink117.runtime - testImplementation(libs.flink117.test.utilsjunit) { + testImplementation libs.flink120.connector.test.utils + testImplementation libs.flink120.core + testImplementation libs.flink120.runtime + testImplementation(libs.flink120.test.utilsjunit) { exclude group: 'junit' } - testImplementation(libs.flink117.test.utils) { + testImplementation(libs.flink120.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -168,7 +168,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink117.metrics.dropwizard + implementation libs.flink120.metrics.dropwizard // for integration testing with the flink-runtime-jar // all of those dependencies are required because the integration test extends FlinkTestBase @@ -178,13 +178,13 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink117.test.utils) { + integrationImplementation(libs.flink120.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } - integrationImplementation libs.flink117.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" + integrationImplementation libs.flink120.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink120.get()}" integrationImplementation libs.hadoop2.common integrationImplementation libs.hadoop2.hdfs diff --git a/flink/v1.17/flink-runtime/LICENSE b/flink/v1.20/flink-runtime/LICENSE similarity index 100% rename from flink/v1.17/flink-runtime/LICENSE rename to flink/v1.20/flink-runtime/LICENSE diff --git a/flink/v1.17/flink-runtime/NOTICE b/flink/v1.20/flink-runtime/NOTICE similarity index 100% rename from flink/v1.17/flink-runtime/NOTICE rename to flink/v1.20/flink-runtime/NOTICE diff --git a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.20/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java similarity index 100% rename from flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java rename to flink/v1.20/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java diff --git a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java similarity index 98% rename from flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java rename to flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index a9ad386a5a4a..592e7ff16241 100644 --- a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -140,8 +140,7 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - // CHARS is all ASCII - return prefix + new String(buffer, StandardCharsets.US_ASCII); + return prefix + new String(buffer, StandardCharsets.UTF_8); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java similarity index 99% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 988465a38f4d..86295d78cc13 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -401,7 +401,6 @@ void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolea Schema icebergSchema = FlinkSchemaUtil.convert(table.getResolvedSchema()); PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); - ImmutableMap.Builder properties = ImmutableMap.builder(); String location = null; for (Map.Entry entry : table.getOptions().entrySet()) { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java similarity index 99% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index 1453753849ec..fe4008a13ce5 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/v1.20/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.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java similarity index 84% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index 3a8f5ccc6c03..4bd85bbd97b4 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -102,22 +102,4 @@ public static RowData clone( return ret; } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java new file mode 100644 index 000000000000..d74b2349b1de --- /dev/null +++ b/flink/v1.20/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.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java new file mode 100644 index 000000000000..20c7684d9700 --- /dev/null +++ b/flink/v1.20/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.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java new file mode 100644 index 000000000000..452ed80ed0e5 --- /dev/null +++ b/flink/v1.20/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.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java similarity index 94% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java index eacef58a8d5d..b6f1392d1562 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java @@ -35,7 +35,6 @@ import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.encryption.EncryptionUtil; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.data.FlinkAvroWriter; import org.apache.iceberg.flink.data.FlinkOrcWriter; @@ -100,11 +99,6 @@ private RowType lazyPosDeleteFlinkSchema() { @Override public FileAppender newAppender(OutputFile outputFile, FileFormat format) { - return newAppender(EncryptionUtil.plainAsEncryptedOutput(outputFile), format); - } - - @Override - public FileAppender newAppender(EncryptedOutputFile outputFile, FileFormat format) { MetricsConfig metricsConfig = MetricsConfig.forTable(table); try { switch (format) { @@ -148,7 +142,7 @@ public FileAppender newAppender(EncryptedOutputFile outputFile, FileFor public DataWriter newDataWriter( EncryptedOutputFile file, FileFormat format, StructLike partition) { return new DataWriter<>( - newAppender(file, format), + newAppender(file.encryptingOutputFile(), format), format, file.encryptingOutputFile().location(), spec, @@ -170,7 +164,7 @@ public EqualityDeleteWriter newEqDeleteWriter( try { switch (format) { case AVRO: - return Avro.writeDeletes(outputFile) + return Avro.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) .withPartition(partition) .overwrite() @@ -183,7 +177,7 @@ public EqualityDeleteWriter newEqDeleteWriter( .buildEqualityWriter(); case ORC: - return ORC.writeDeletes(outputFile) + return ORC.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc( (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) .withPartition(partition) @@ -197,7 +191,7 @@ public EqualityDeleteWriter newEqDeleteWriter( .buildEqualityWriter(); case PARQUET: - return Parquet.writeDeletes(outputFile) + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc( msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) .withPartition(partition) @@ -226,7 +220,7 @@ public PositionDeleteWriter newPosDeleteWriter( try { switch (format) { case AVRO: - return Avro.writeDeletes(outputFile) + return Avro.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) .withPartition(partition) .overwrite() @@ -240,7 +234,7 @@ public PositionDeleteWriter newPosDeleteWriter( case ORC: RowType orcPosDeleteSchema = FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return ORC.writeDeletes(outputFile) + return ORC.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc( (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) .withPartition(partition) @@ -256,7 +250,7 @@ public PositionDeleteWriter newPosDeleteWriter( case PARQUET: RowType flinkPosDeleteSchema = FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(outputFile) + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc( msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) .withPartition(partition) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java similarity index 82% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java index 9de0d6aaa551..5b28c4acb1c5 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java @@ -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.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java similarity index 93% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java similarity index 95% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index c0e228965ddd..e4cba174f0f2 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java similarity index 98% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java index 7f55188e7f8c..1ac0e386a011 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -148,9 +148,7 @@ public static class CompletedStatisticsSerializerSnapshot /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public CompletedStatisticsSerializerSnapshot() { - super(CompletedStatisticsSerializer.class); - } + public CompletedStatisticsSerializerSnapshot() {} @SuppressWarnings("checkstyle:RedundantModifier") public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java similarity index 92% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 3b21fbae315a..4bfde7204acf 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java similarity index 97% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 59c38b239725..7995a8a5b181 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -92,8 +92,8 @@ public class DataStatisticsOperator extends AbstractStreamOperator keyAssignments() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java similarity index 94% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java similarity index 77% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java similarity index 91% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java similarity index 92% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index d1d75019fa2e..b82fc8250763 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -27,7 +27,6 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -317,15 +316,14 @@ 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(); } // Sort order should be identical - SortKeySerializerSnapshot newSnapshot = - (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); - if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { + SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; + if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } @@ -333,10 +331,10 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); // only care about the schema related to sort fields Schema sortSchema = TypeUtil.project(schema, sortFieldIds); - Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); + Schema oldSortSchema = TypeUtil.project(oldSnapshot.schema, sortFieldIds); List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); + CheckCompatibility.writeCompatibilityErrors(sortSchema, oldSortSchema); if (compatibilityErrors.isEmpty()) { return TypeSerializerSchemaCompatibility.compatibleAsIs(); } @@ -357,17 +355,5 @@ private void readV1(DataInputView in) throws IOException { this.schema = SchemaParser.fromJson(schemaJson); this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); } - - @VisibleForTesting - static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 98% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index 6e403425938d..d4ae2b359679 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -174,9 +174,7 @@ public static class StatisticsOrRecordSerializerSnapshot /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public StatisticsOrRecordSerializerSnapshot() { - super(StatisticsOrRecordSerializer.class); - } + public StatisticsOrRecordSerializerSnapshot() {} @SuppressWarnings("checkstyle:RedundantModifier") public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java similarity index 99% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..ccbd0d9997ed 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 95% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java index 37a0f1a6055f..e7447d08c985 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java @@ -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.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java similarity index 95% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java index ca60612f0ec9..dae7c8cca70c 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java @@ -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.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java similarity index 88% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 3aca390755ed..280a126a46ce 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -36,12 +37,9 @@ 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 { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -55,6 +53,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 @@ -93,6 +97,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java similarity index 94% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index 55451b105885..c7021b9c6847 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -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.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java similarity index 76% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java new file mode 100644 index 000000000000..6306e82d5729 --- /dev/null +++ b/flink/v1.20/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.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java similarity index 100% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java similarity index 88% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java index 91ed3c4adea3..062ff68d5d85 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -118,26 +118,4 @@ protected String getFullQualifiedTableName(String tableName) { static String getURI(HiveConf conf) { return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } + + protected static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestBase.java similarity index 81% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..401960c3591b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink; +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; import static org.assertj.core.api.Assertions.assertThat; import java.nio.file.Path; @@ -26,7 +27,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -40,10 +40,10 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -public abstract class TestBase extends TestBaseUtils { +public abstract class TestBase extends SqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; @@ -71,6 +71,7 @@ public static void stopMetastore() throws Exception { TestBase.catalog = null; } + @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { synchronized (this) { @@ -122,7 +123,23 @@ protected void assertSameElements(String message, Iterable expected, Iterab * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog */ protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java index 7fc6ab82490d..1997ef6998a2 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -61,7 +61,7 @@ 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; @Parameter private boolean partitioned; @@ -92,7 +92,7 @@ public void before() throws IOException { @Override public void clean() { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); BoundedTableFactory.clearDataSets(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java similarity index 97% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index 6850060e8fc8..bd07087756ad 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.20/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(); - assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + assertThatThrownBy(() -> dropDatabase(flinkDatabase, true)) .cause() .isInstanceOf(DatabaseNotEmptyException.class) .hasMessage( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 7a364b856398..0b9c2193b4d5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -73,7 +73,7 @@ public void before() { public void cleanNamespaces() { sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index e395414e925d..e69e1ac4d713 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -76,7 +76,7 @@ public void before() { @AfterEach public void cleanNamespaces() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java index f1de267cf29b..91343ab1ee72 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java @@ -95,7 +95,7 @@ private void checkSQLQuery(Map catalogProperties, File warehouse .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.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 63% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..2978a92945a2 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -18,36 +18,21 @@ */ package org.apache.iceberg.flink; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -91,7 +76,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); @@ -122,7 +107,7 @@ public void before() { @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); BoundedTableFactory.clearDataSets(); super.clean(); } @@ -169,39 +154,6 @@ public void testOverwriteTable() throws Exception { icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); } - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); - String insertSQL = - String.format( - "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, SOURCE_TABLE); - ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); - Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); - Transformation committer = dummySink.getInputs().get(0); - Transformation writer = committer.getInputs().get(0); - - assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); - writer - .getInputs() - .forEach( - input -> - assertThat(input.getParallelism()) - .as("Should have the expected parallelism.") - .isEqualTo(isStreamingJob ? 2 : 4)); - } - @TestTemplate public void testReplacePartitions() throws Exception { assumeThat(isStreamingJob) @@ -289,70 +241,4 @@ public void testInsertIntoPartition() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); } } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java new file mode 100644 index 000000000000..482cfd110bde --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List parameters() { + return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + } + + protected synchronized TableEnvironment getTableEnv() { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + return tEnv; + } + + @BeforeEach + public void before() { + String warehouseLocation = "file:" + warehouseRoot.getPath(); + this.catalog = new HadoopCatalog(new Configuration(), warehouseLocation); + Map config = Maps.newHashMap(); + config.put("type", "iceberg"); + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HADOOP); + config.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + sql("CREATE CATALOG %s WITH %s", CATALOG, toWithClause(config)); + + sql("CREATE DATABASE %s", FLINK_DATABASE); + sql("USE CATALOG %s", CATALOG); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE, FileFormat.PARQUET.name()); + } + + @AfterEach + public void clean() throws Exception { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, TABLE); + dropDatabase(FLINK_DATABASE, true); + BoundedTableFactory.clearDataSets(); + + dropCatalog(CATALOG, true); + catalog.close(); + } + + @TestTemplate + public void testWriteParallelism() { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) getTableEnv()).getPlanner(); + String insertSQL = + String.format( + "INSERT INTO %s /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", + TABLE, SOURCE_TABLE); + ModifyOperation operation = (ModifyOperation) planner.getParser().parse(insertSQL).get(0); + Transformation dummySink = planner.translate(Collections.singletonList(operation)).get(0); + Transformation committer = dummySink.getInputs().get(0); + Transformation writer = committer.getInputs().get(0); + + assertThat(writer.getParallelism()).as("Should have the expected 1 parallelism.").isEqualTo(1); + writer + .getInputs() + .forEach( + input -> + assertThat(input.getParallelism()) + .as("Should have the expected parallelism.") + .isEqualTo(isStreamingJob ? 2 : 4)); + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..c5becb6caca1 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); @@ -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.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java similarity index 93% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 013b98e3b82b..47f5485df879 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); @@ -248,20 +248,12 @@ public void testCreateTableUnderDefaultDatabase() { public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); } @TestTemplate @@ -327,10 +319,6 @@ private String databaseName() { return properties.getOrDefault("catalog-database", "default_database"); } - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - private String createWarehouse() { try { return String.format( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 800cce96edac..6b8399f666d4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -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.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java index 91ee017238ac..72f2ce4f4bce 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java @@ -70,7 +70,7 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i++) { - assertThat(rows.hasNext()).isTrue(); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } assertThat(rows).isExhausted(); @@ -98,7 +98,7 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(records).hasNext(); + assertThat(records.hasNext()).isTrue(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } assertThat(records).isExhausted(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index e76452b7cea0..7dd4e8759c0e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -39,6 +39,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -211,6 +212,7 @@ public void testBasicProjection() throws Exception { projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); assertThat(projected.getString(0)).asString().isEqualTo("test"); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java new file mode 100644 index 000000000000..9b6580fad0bf --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Sink for collecting output during testing. */ +class CollectingSink implements Sink { + private static final long serialVersionUID = 1L; + private static final List> QUEUES = + Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); + private static final AtomicInteger NUM_SINKS = new AtomicInteger(-1); + private final int index; + + /** Creates a new sink which collects the elements received. */ + CollectingSink() { + this.index = NUM_SINKS.incrementAndGet(); + QUEUES.add(new LinkedBlockingQueue<>()); + } + + /** + * Gets all the remaining output received by this {@link Sink}. + * + * @return all the remaining output + */ + List remainingOutput() { + return Lists.newArrayList((BlockingQueue) QUEUES.get(this.index)); + } + + /** + * Check if there is no remaining output received by this {@link Sink}. + * + * @return true if there is no remaining output + */ + boolean isEmpty() { + return QUEUES.get(this.index).isEmpty(); + } + + /** + * Wait until the next element received by the {@link Sink}. + * + * @param timeout for the poll + * @return The first element received by this {@link Sink} + * @throws TimeoutException if no element received until the timeout + */ + T poll(Duration timeout) throws TimeoutException { + Object element; + + try { + element = QUEUES.get(this.index).poll(timeout.toMillis(), TimeUnit.MILLISECONDS); + } catch (InterruptedException var4) { + throw new RuntimeException(var4); + } + + if (element == null) { + throw new TimeoutException(); + } else { + return (T) element; + } + } + + @Override + public SinkWriter createWriter(InitContext context) { + return new CollectingWriter<>(index); + } + + private static class CollectingWriter implements SinkWriter { + private final int index; + + CollectingWriter(int index) { + this.index = index; + } + + @Override + public void write(T element, Context context) { + QUEUES.get(index).add(element); + } + + @Override + public void flush(boolean endOfInput) { + // Nothing to do here + } + + @Override + public void close() { + // Nothing to do here + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java new file mode 100644 index 000000000000..91d36aa3e85d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +/** + * Junit 5 extension for running Flink SQL queries. {@link + * org.apache.flink.test.junit5.MiniClusterExtension} is used for executing the SQL batch jobs. + */ +public class FlinkSqlExtension implements BeforeEachCallback, AfterEachCallback { + private final String catalogName; + private final Map catalogProperties; + private final String databaseName; + private final Path warehouse; + private final CatalogLoader catalogLoader; + private TableEnvironment tableEnvironment; + + public FlinkSqlExtension( + String catalogName, Map catalogProperties, String databaseName) { + this.catalogName = catalogName; + this.catalogProperties = Maps.newHashMap(catalogProperties); + this.databaseName = databaseName; + + // Add temporary dir as a warehouse location + try { + this.warehouse = Files.createTempDirectory("warehouse"); + } catch (IOException e) { + throw new RuntimeException(e); + } + this.catalogProperties.put( + CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouse)); + this.catalogLoader = + CatalogLoader.hadoop(catalogName, new Configuration(), this.catalogProperties); + } + + @Override + public void beforeEach(ExtensionContext context) { + // We need to recreate the tableEnvironment for every test as the minicluster is recreated + this.tableEnvironment = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + exec("CREATE CATALOG %s WITH %s", catalogName, toWithClause(catalogProperties)); + exec("CREATE DATABASE %s.%s", catalogName, databaseName); + exec("USE CATALOG %s", catalogName); + exec("USE %s", databaseName); + } + + @Override + public void afterEach(ExtensionContext context) throws IOException { + List tables = exec("SHOW TABLES"); + tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0))); + exec("USE CATALOG default_catalog"); + exec("DROP CATALOG IF EXISTS %s", catalogName); + try (Stream files = Files.walk(warehouse)) { + files.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } + } + + /** + * Executes an SQL query with the given parameters. The parameter substitution is done by {@link + * String#format(String, Object...)}. + * + * @param query to run + * @param parameters to substitute to the query + * @return The {@link Row}s returned by the query + */ + public List exec(String query, Object... parameters) { + TableResult tableResult = tableEnvironment.executeSql(String.format(query, parameters)); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + /** + * Returns the {@link TableLoader} which could be used to access the given table. + * + * @param tableName of the table + * @return the {@link TableLoader} for the table + */ + public TableLoader tableLoader(String tableName) { + TableLoader tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(databaseName, tableName)); + tableLoader.open(); + return tableLoader; + } + + private static String toWithClause(Map props) { + return String.format( + "(%s)", + props.entrySet().stream() + .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(","))); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java new file mode 100644 index 000000000000..9cdc55cb0cce --- /dev/null +++ b/flink/v1.20/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.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java new file mode 100644 index 000000000000..38bb9c393fa9 --- /dev/null +++ b/flink/v1.20/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.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java new file mode 100644 index 000000000000..272e0b693fd3 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.extension.RegisterExtension; + +class OperatorTestBase { + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + static final String TABLE_NAME = "test_table"; + + @RegisterExtension + protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .build()); + + @RegisterExtension + final FlinkSqlExtension sql = + new FlinkSqlExtension( + "catalog", + ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), + "db"); +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java new file mode 100644 index 000000000000..876d64214560 --- /dev/null +++ b/flink/v1.20/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.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java new file mode 100644 index 000000000000..b778037c559c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + public void testWriteRow() throws Exception { + testWriteRow(parallelism, null, DistributionMode.NONE); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java similarity index 58% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index b38aa6b50ce6..9ce36cc1e8d0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -18,29 +18,52 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkIcebergSinkBase { - protected Table table; - protected StreamExecutionEnvironment env; + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + protected static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); protected static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List createRows(String prefix) { protected List convertToRowData(List rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java similarity index 97% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java new file mode 100644 index 000000000000..75e397d3f203 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * This tests the distribution mode of Flink sink. Extract them separately since it is unnecessary + * to test different file formats (Avro, Orc, Parquet) like in {@link TestFlinkIcebergSink}. + * Removing the file format dimension reduces the number of combinations from 12 to 4, which helps + * reduce test run time. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private final FileFormat format = FileFormat.PARQUET; + + @Parameter(index = 0) + private int parallelism; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "parallelism = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {1, true}, + {1, false}, + {2, true}, + {2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).isGreaterThan(3); + } + } + } + + @TestTemplate + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testPartitionWriteMode() throws Exception { + testWriteRow(parallelism, null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); + rightTable.refresh(); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java similarity index 97% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java similarity index 96% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java similarity index 87% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index f79a3e634071..939ed2be7dbc 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -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.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index c760f1ba96d3..bc248b778184 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -39,6 +39,7 @@ 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; @@ -332,7 +333,8 @@ private StateInitializationContext getStateContext() throws Exception { CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); OperatorStateStore operatorStateStore = abstractStateBackend.createOperatorStateBackend( - env, "test-operator", Collections.emptyList(), cancelStreamRegistry); + new OperatorStateBackendParametersImpl( + env, "test-operator", Collections.emptyList(), cancelStreamRegistry)); return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
    + *
  • age <= 15 + *
  • age > 15 && age <= 32 + *
  • age >32 && age <= 60 + *
  • age > 60 + *
+ */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java similarity index 63% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java similarity index 79% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java index 012654603b04..c0f688f2589e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -75,22 +75,21 @@ public void testRestoredSerializer() throws Exception { @Test public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); - SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); } @Test public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); // removed non-sort boolean field Schema newSchema = @@ -99,18 +98,18 @@ public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { Types.NestedField.optional(2, "str", Types.StringType.get()), Types.NestedField.optional(3, "int", Types.IntegerType.get())); SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); - SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); } @Test public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); // add a new non-sort float field Schema newSchema = @@ -121,18 +120,18 @@ public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), Types.NestedField.required(5, "float", Types.FloatType.get())); SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); - SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); } @Test public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); // change str field to a long type Schema newSchema = @@ -143,56 +142,57 @@ public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); // switch sort field order - SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isIncompatible()).isTrue(); } @Test public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); // removed str field from sort order SortOrder newSortOrder = SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); - SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isIncompatible()).isTrue(); } @Test public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); // remove str field from sort order SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); - SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isIncompatible()).isTrue(); } @Test public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); // switch sort field order SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); - SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); + newSnapshot.resolveSchemaCompatibility(oldSnapshot); assertThat(resultCompatibility.isIncompatible()).isTrue(); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java similarity index 95% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java index 32c81d9465a4..f89d63ac73e3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -99,6 +99,7 @@ public void before() throws IOException { @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java index 226da5813ad8..1b4fc863631f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java @@ -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.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java similarity index 95% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java index 5be4a31b4ac8..59a4c3118cdf 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TestMergingMetrics; @@ -57,8 +58,7 @@ protected FileAppender writeAndGetAppender(List records) throws null, null) .newAppender( - org.apache.iceberg.Files.localOutput(File.createTempFile("junit", null, tempDir)), - fileFormat); + Files.localOutput(File.createTempFile("junit", null, tempDir)), fileFormat); try (FileAppender fileAppender = appender) { records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index f58cc87c6a29..8352924d042a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -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.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java similarity index 92% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java similarity index 93% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 9c7006e16b8e..749cbf89338a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -25,6 +25,7 @@ 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; @@ -32,7 +33,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; @@ -59,11 +62,13 @@ public class TestIcebergSourceContinuous { + public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); + @TempDir protected Path temporaryFolder; @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); @RegisterExtension private static final HadoopTableExtension TABLE_EXTENSION = @@ -113,6 +118,8 @@ public void testTableScanThenIncremental() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -165,6 +172,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -216,6 +225,8 @@ public void testEarliestSnapshot() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -271,6 +282,8 @@ public void testLatestSnapshot(@InjectClusterClient ClusterClient clusterClie List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -323,6 +336,8 @@ public void testSpecificSnapshotId() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -379,6 +394,8 @@ public void testSpecificSnapshotTimestamp() throws Exception { List result3 = waitForResult(iter, 2); TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -535,4 +552,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.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..992b712d9d69 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), + getRuntimeContext().getTaskInfo().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, Duration.ofSeconds(5)); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java index 40dfda723749..9cf953342a18 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -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.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java similarity index 98% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d6cf6791270e..97ed4ca1e93f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -55,7 +55,7 @@ public class TestStreamScanSql extends CatalogTestBase { private static final String TABLE = "test_table"; private static final FileFormat FORMAT = FileFormat.PARQUET; - private TableEnvironment tEnv; + private volatile TableEnvironment tEnv; @Override protected TableEnvironment getTableEnv() { @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = @@ -98,7 +98,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.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java similarity index 99% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 8bd1214bd960..af806d4c655d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -74,7 +74,7 @@ public class TestColumnStatsWatermarkExtractor { @TempDir protected Path temporaryFolder; @RegisterExtension - private static final HadoopTableExtension SOURCE_TABLE_RESOURCE = + private static final HadoopTableExtension SOURCE_TABLE_EXTENSION = new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); @Parameter(index = 0) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java similarity index 97% rename from flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index 079c70bae070..65f21f7d050c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -29,7 +29,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - assertThat(FlinkPackage.version()).isEqualTo("1.17.2"); + assertThat(FlinkPackage.version()).isEqualTo("1.20.0"); } @Test diff --git a/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/format/spec.md b/format/spec.md index 5a90f6fd978d..c322f8174fe2 100644 --- a/format/spec.md +++ b/format/spec.md @@ -150,6 +150,10 @@ Readers should be more permissive because v1 metadata files are allowed in v2 ta Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. +##### Writing data files + +All columns must be written to data files even if they introduce redundancy with metadata stored in manifest files (e.g. columns with identity partition transforms). Writing all columns provides a backup in case of corruption or bugs in the metadata layer. + ### Schemas and Data Types A table's **schema** is a list of named columns. All data types are either primitives or nested types, which are maps, lists, or structs. A table schema is also a struct type. @@ -241,7 +245,14 @@ Struct evolution requires the following rules for default values: #### Column Projection -Columns in Iceberg data files are selected by field id. The table schema's column names and order may change after a data file is written, and projection must be done using field ids. If a field id is missing from a data file, its value for each row should be `null`. +Columns in Iceberg data files are selected by field id. The table schema's column names and order may change after a data file is written, and projection must be done using field ids. + +Values for field ids which are not present in a data file must be resolved according the following rules: + +* Return the value from partition metadata if an [Identity Transform](#partition-transforms) exists for the field and the partition value is present in the `partition` struct on `data_file` object in the manifest. This allows for metadata only migrations of Hive tables. +* Use `schema.name-mapping.default` metadata to map field id to columns without field id as described below and use the column if it is present. +* Return the default value if it has a defined `initial-default` (See [Default values](#default-values) section for more details). +* Return `null` in all other cases. For example, a file may be written with schema `1: a int, 2: b string, 3: c double` and read using projection schema `3: measurement, 2: name, 4: a`. This must select file columns `c` (renamed to `measurement`), `b` (now called `name`), and a column of `null` values called `a`; in that order. @@ -759,13 +770,13 @@ The unified partition type is a struct containing all fields that have ever been and sorted by the field ids in ascending order. In other words, the struct fields represent a union of all known partition fields sorted in ascending order by the field ids. For example, -1) spec#0 has two fields {field#1, field#2} -and then the table has evolved into spec#1 which has three fields {field#1, field#2, field#3}. -The unified partition type looks like Struct. +1) `spec#0` has two fields `{field#1, field#2}` +and then the table has evolved into `spec#1` which has three fields `{field#1, field#2, field#3}`. +The unified partition type looks like `Struct`. -2) spec#0 has two fields {field#1, field#2} -and then the table has evolved into spec#1 which has just one field {field#2}. -The unified partition type looks like Struct. +2) `spec#0` has two fields `{field#1, field#2}` +and then the table has evolved into `spec#1` which has just one field `{field#2}`. +The unified partition type looks like `Struct`. #### Commit Conflict Resolution and Retry @@ -779,7 +790,9 @@ When two commits happen at the same time and are based on the same version, only #### File System Tables -An atomic swap can be implemented using atomic rename in file systems that support it, like HDFS or most local file systems [1]. +_Note: This file system based scheme to commit a metadata file is **deprecated** and will be removed in version 4 of this spec. The scheme is **unsafe** in object stores and local file systems._ + +An atomic swap can be implemented using atomic rename in file systems that support it, like HDFS [1]. Each version of table metadata is stored in a metadata folder under the table’s base location using a file naming scheme that includes a version number, `V`: `v.metadata.json`. To commit a new metadata version, `V+1`, the writer performs the following steps: @@ -1393,4 +1406,4 @@ This section covers topics not required by the specification but recommendations Iceberg supports two types of histories for tables. A history of previous "current snapshots" stored in ["snapshot-log" table metadata](#table-metadata-fields) and [parent-child lineage stored in "snapshots"](#table-metadata-fields). These two histories might indicate different snapshot IDs for a specific timestamp. The discrepancies can be caused by a variety of table operations (e.g. updating the `current-snapshot-id` can be used to set the snapshot of a table to any arbitrary snapshot, which might have a lineage derived from a table branch or no lineage at all). -When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. \ No newline at end of file +When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. diff --git a/gradle.properties b/gradle.properties index c6b8dec17bc5..fcbe7d8de012 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,8 +16,8 @@ jmhOutputPath=build/reports/jmh/human-readable-output.txt jmhJsonOutputPath=build/reports/jmh/results.json jmhIncludeRegex=.* -systemProp.defaultFlinkVersions=1.19 -systemProp.knownFlinkVersions=1.17,1.18,1.19 +systemProp.defaultFlinkVersions=1.20 +systemProp.knownFlinkVersions=1.18,1.19,1.20 systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 055475809296..1cc38d44ac9d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -25,10 +25,10 @@ aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.27" arrow = "15.0.2" -avro = "1.11.3" +avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.1" -awssdk-bom = "2.26.25" +awssdk-bom = "2.26.29" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" @@ -39,9 +39,9 @@ delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.29.2" findbugs-jsr305 = "3.0.2" -flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} +flink120 = { strictly = "1.20.0"} google-libraries-bom = "26.43.0" guava = "33.2.1-jre" hadoop2 = "2.7.3" @@ -57,16 +57,17 @@ jackson213 = { strictly = "2.13.4"} jackson214 = { strictly = "2.14.2"} jackson215 = { strictly = "2.15.2"} jakarta-el-api = "3.0.3" +jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" -jetty = "9.4.55.v20240627" +jetty = "11.0.22" junit = "5.10.1" -kafka = "3.7.1" +kafka = "3.8.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.94.2" +nessie = "0.94.4" netty-buffer = "4.1.112.Final" netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" @@ -74,17 +75,17 @@ orc = "1.9.4" parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.2.1" -s3mock-junit5 = "2.11.0" +s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" snowflake-jdbc = "3.18.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" -spark-hive35 = "3.5.1" +spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.37" -sqlite-jdbc = "3.46.0.0" -testcontainers = "1.20.0" +sqlite-jdbc = "3.46.0.1" +testcontainers = "1.20.1" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above @@ -107,12 +108,6 @@ datasketches = { module = "org.apache.datasketches:datasketches-java", version.r delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } -flink117-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink117" } -flink117-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink117" } -flink117-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink117" } -flink117-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink117" } -flink117-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink117" } -flink117-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink117" } flink118-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink118" } flink118-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink118" } flink118-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink118" } @@ -125,6 +120,12 @@ flink119-connector-files = { module = "org.apache.flink:flink-connector-files", flink119-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink119" } flink119-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink119" } flink119-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink119" } +flink120-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink120" } +flink120-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink120" } +flink120-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink120" } +flink120-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink120" } +flink120-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink120" } +flink120-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink120" } google-libraries-bom = { module = "com.google.cloud:libraries-bom", version.ref = "google-libraries-bom" } guava-guava = { module = "com.google.guava:guava", version.ref = "guava" } hadoop2-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop2" } @@ -179,11 +180,6 @@ assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-cor awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } -flink117-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink117" } -flink117-core = { module = "org.apache.flink:flink-core", version.ref = "flink117" } -flink117-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink117" } -flink117-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink117" } -flink117-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink117" } flink118-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink118" } flink118-core = { module = "org.apache.flink:flink-core", version.ref = "flink118" } flink118-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink118" } @@ -194,8 +190,14 @@ flink119-core = { module = "org.apache.flink:flink-core", version.ref = "flink11 flink119-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink119" } flink119-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink119" } flink119-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink119" } +flink120-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink120" } +flink120-core = { module = "org.apache.flink:flink-core", version.ref = "flink120" } +flink120-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink120" } +flink120-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink120" } +flink120-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink120" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jakarta-el-api = { module = "jakarta.el:jakarta.el-api", version.ref = "jakarta-el-api" } +jakarta-servlet = {module = "jakarta.servlet:jakarta.servlet-api", version.ref = "jakarta-servlet-api"} jetty-server = { module = "org.eclipse.jetty:jetty-server", version.ref = "jetty" } jetty-servlet = { module = "org.eclipse.jetty:jetty-servlet", version.ref = "jetty" } junit-jupiter = { module = "org.junit.jupiter:junit-jupiter", version.ref = "junit" } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java index 055a14246e77..6500e724a4f0 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -31,7 +31,6 @@ import org.apache.iceberg.ClientPool; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.io.FileIO; @@ -150,15 +149,6 @@ default void persistTable(Table hmsTable, boolean updateHiveTable, String metada } } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link #storageDescriptor(Schema, - * String, boolean)} instead - */ - @Deprecated - static StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { - return storageDescriptor(metadata.schema(), metadata.location(), hiveEngineEnabled); - } - static StorageDescriptor storageDescriptor( Schema schema, String location, boolean hiveEngineEnabled) { final StorageDescriptor storageDescriptor = new StorageDescriptor(); diff --git a/hive-runtime/build.gradle b/hive-runtime/build.gradle index b2051d529512..6f891be8087e 100644 --- a/hive-runtime/build.gradle +++ b/hive-runtime/build.gradle @@ -38,7 +38,7 @@ project(':iceberg-hive-runtime') { dependencies { implementation project(':iceberg-mr') - if (jdkVersion == '8' && hiveVersions.contains("3")) { + if (hiveVersions.contains("3")) { implementation project(':iceberg-hive3') } implementation(project(':iceberg-nessie')) { diff --git a/jmh.gradle b/jmh.gradle index 80f5f8d0ea63..a5d8d624270d 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (jdkVersion != '8' && jdkVersion != '11' && jdkVersion != '17' && jdkVersion != '21') { - throw new GradleException("The JMH benchmarks must be run with JDK 8 or JDK 11 or JDK 17 or JDK 21") +if (jdkVersion != '11' && jdkVersion != '17' && jdkVersion != '21') { + throw new GradleException("The JMH benchmarks must be run with JDK 11 or JDK 17 or JDK 21") } def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") @@ -26,16 +26,16 @@ def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getPro def scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") def jmhProjects = [project(":iceberg-core"), project(":iceberg-data")] -if (flinkVersions.contains("1.16")) { - jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.16")) +if (flinkVersions.contains("1.18")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.18")) } -if (flinkVersions.contains("1.17")) { - jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.17")) +if (flinkVersions.contains("1.19")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.19")) } -if (flinkVersions.contains("1.18")) { - jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.18")) +if (flinkVersions.contains("1.20")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.20")) } if (sparkVersions.contains("3.3")) { diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 60fa879d291f..a0e55f23405d 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -71,7 +71,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' force 'org.xerial.snappy:snappy-java:1.1.10.5' - force 'org.apache.commons:commons-compress:1.26.0' + force 'org.apache.commons:commons-compress:1.26.2' force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' } } diff --git a/settings.gradle b/settings.gradle index cdc69b0e2071..1e6d92bf1e1f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -112,15 +112,6 @@ if (!flinkVersions.isEmpty()) { project(':flink').name = 'iceberg-flink' } -if (flinkVersions.contains("1.17")) { - include ":iceberg-flink:flink-1.17" - include ":iceberg-flink:flink-runtime-1.17" - project(":iceberg-flink:flink-1.17").projectDir = file('flink/v1.17/flink') - project(":iceberg-flink:flink-1.17").name = "iceberg-flink-1.17" - project(":iceberg-flink:flink-runtime-1.17").projectDir = file('flink/v1.17/flink-runtime') - project(":iceberg-flink:flink-runtime-1.17").name = "iceberg-flink-runtime-1.17" -} - if (flinkVersions.contains("1.18")) { include ":iceberg-flink:flink-1.18" include ":iceberg-flink:flink-runtime-1.18" @@ -139,6 +130,15 @@ if (flinkVersions.contains("1.19")) { project(":iceberg-flink:flink-runtime-1.19").name = "iceberg-flink-runtime-1.19" } +if (flinkVersions.contains("1.20")) { + include ":iceberg-flink:flink-1.20" + include ":iceberg-flink:flink-runtime-1.20" + project(":iceberg-flink:flink-1.20").projectDir = file('flink/v1.20/flink') + project(":iceberg-flink:flink-1.20").name = "iceberg-flink-1.20" + project(":iceberg-flink:flink-runtime-1.20").projectDir = file('flink/v1.20/flink-runtime') + project(":iceberg-flink:flink-runtime-1.20").name = "iceberg-flink-runtime-1.20" +} + if (sparkVersions.contains("3.3")) { include ":iceberg-spark:spark-3.3_${scalaVersion}" include ":iceberg-spark:spark-extensions-3.3_${scalaVersion}" diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 88a14e7153b4..60bc89f9537f 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -84,7 +84,7 @@ settle disagreements or to force a decision. ## Building the Project Locally -Iceberg is built using Gradle with Java 8, 11, 17, or 21. +Iceberg is built using Gradle with Java 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index e4123cc579b3..a3c63276bfdb 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -90,9 +90,10 @@ Users should continuously upgrade their Flink version to stay up-to-date. | 1.14 | End of Life | 0.13.0 | 1.2.0 | [iceberg-flink-runtime-1.14](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.14/1.2.0/iceberg-flink-runtime-1.14-1.2.0.jar) | | 1.15 | End of Life | 0.14.0 | 1.4.3 | [iceberg-flink-runtime-1.15](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.15/1.4.3/iceberg-flink-runtime-1.15-1.4.3.jar) | | 1.16 | End of Life | 1.1.0 | 1.5.0 | [iceberg-flink-runtime-1.16](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/1.5.0/iceberg-flink-runtime-1.16-1.5.0.jar) | -| 1.17 | Deprecated | 1.3.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.17](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) | +| 1.17 | Deprecated | 1.3.0 | 1.6.0 | [iceberg-flink-runtime-1.17](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) | | 1.18 | Maintained | 1.5.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.18](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) | | 1.19 | Maintained | 1.6.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.19](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.19/{{ icebergVersion }}/iceberg-flink-runtime-1.19-{{ icebergVersion }}.jar) | +| 1.20 | Maintained | 1.7.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.20](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.20/{{ icebergVersion }}/iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar) | diff --git a/site/docs/releases.md b/site/docs/releases.md index 73d8bc32a071..6b48e31a0728 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -31,9 +31,9 @@ The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apach * [{{ icebergVersion }} Spark 3.4\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.4_2.13-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Spark 3.3\_with Scala 2.12 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.12-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Spark 3.3\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.13-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Flink 1.20 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.20/{{ icebergVersion }}/iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Flink 1.19 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.19/{{ icebergVersion }}/iceberg-flink-runtime-1.19-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.18 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Flink 1.17 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Flink 1.16 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Hive runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-hive-runtime/{{ icebergVersion }}/iceberg-hive-runtime-{{ icebergVersion }}.jar) * [{{ icebergVersion }} aws-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-aws-bundle/{{ icebergVersion }}/iceberg-aws-bundle-{{ icebergVersion }}.jar) * [{{ icebergVersion }} gcp-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-gcp-bundle/{{ icebergVersion }}/iceberg-gcp-bundle-{{ icebergVersion }}.jar) @@ -83,7 +83,7 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf - Add EnvironmentContext to commit summary ([\#9273](https://github.com/apache/iceberg/pull/9273)) - Add explicit JSON parser for ConfigResponse ([\#9952](https://github.com/apache/iceberg/pull/9952)) - Calling rewrite_position_delete_files fails on tables with more than 1k columns ([\#10020](https://github.com/apache/iceberg/pull/10020)) - - Expose table incremental scan for appends API in SerializableTable ([\#10682](Expose table incremental scan for appends API in SerializableTable)) + - Expose table incremental scan for appends API in SerializableTable ([\#10682](https://github.com/apache/iceberg/pull/10682)) - Fix NPE during conflict handling of NULL partitions ([\#10680](https://github.com/apache/iceberg/pull/10680)) - Fix ParallelIterable memory leak where queue continues to be populated even after iterator close ([\#9402](https://github.com/apache/iceberg/pull/9402)) - Fix logging table name in scanning metadata table ([\#10141](https://github.com/apache/iceberg/pull/10141)) @@ -100,7 +100,7 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf - REST Catalog: Handles possible heap data corruption of OAuth2Util.AuthSession#headers ([\#10615](https://github.com/apache/iceberg/pull/10615)) - REST Catalog: Handles potential NPE in RESTSessionCatalog#newSessionCache ([\#10607](https://github.com/apache/iceberg/pull/10607)) - REST Catalog: Introduce AuthConfig ([\#10161](https://github.com/apache/iceberg/pull/10161)) - - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](Mark 502 and 504 statuses as retryable to the REST exponential retry strategy)) + - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](https://github.com/apache/iceberg/pull/9885)) - REST Catalog: disallow overriding "credential" in table sessions ([\#10345](https://github.com/apache/iceberg/pull/10345)) - REST Catalog: fix incorrect token refresh thread name ([\#10223](https://github.com/apache/iceberg/pull/10223)) - REST Catalog: fix spurious warning when shutting down refresh executor ([\#10087](https://github.com/apache/iceberg/pull/10087)) diff --git a/site/requirements.txt b/site/requirements.txt index 3fd9cf45e443..cb85511b7d96 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.30 +mkdocs-material==9.5.31 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.1 diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index bd29fb0d6d42..67e9d78ada4d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -347,4 +347,12 @@ private boolean executorCacheLocalityEnabledInternal() { .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT) .parse(); } + + public boolean reportColumnStats() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.REPORT_COLUMN_STATS) + .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) + .parse(); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index ea8f6fe0718b..77ae796ffb76 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -90,4 +90,8 @@ private SparkSQLProperties() {} public static final String EXECUTOR_CACHE_LOCALITY_ENABLED = "spark.sql.iceberg.executor-cache.locality.enabled"; public static final boolean EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT = false; + + // Controls whether to report available column statistics to Spark for query optimization. + public static final String REPORT_COLUMN_STATS = "spark.sql.iceberg.report-column-stats"; + public static final boolean REPORT_COLUMN_STATS_DEFAULT = true; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java index 7cde3e1fbe11..71b53d70262f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java @@ -88,7 +88,7 @@ class SparkChangelogScan implements Scan, SupportsReportStatistics { public Statistics estimateStatistics() { long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount); + return new Stats(sizeInBytes, rowsCount, Collections.emptyMap()); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java new file mode 100644 index 000000000000..faaff3631d7c --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.util.Optional; +import java.util.OptionalLong; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.connector.read.colstats.Histogram; + +class SparkColumnStatistics implements ColumnStatistics { + + private final OptionalLong distinctCount; + private final Optional min; + private final Optional max; + private final OptionalLong nullCount; + private final OptionalLong avgLen; + private final OptionalLong maxLen; + private final Optional histogram; + + SparkColumnStatistics( + Long distinctCount, + Object min, + Object max, + Long nullCount, + Long avgLen, + Long maxLen, + Histogram histogram) { + this.distinctCount = + (distinctCount == null) ? OptionalLong.empty() : OptionalLong.of(distinctCount); + this.min = Optional.ofNullable(min); + this.max = Optional.ofNullable(max); + this.nullCount = (nullCount == null) ? OptionalLong.empty() : OptionalLong.of(nullCount); + this.avgLen = (avgLen == null) ? OptionalLong.empty() : OptionalLong.of(avgLen); + this.maxLen = (maxLen == null) ? OptionalLong.empty() : OptionalLong.of(maxLen); + this.histogram = Optional.ofNullable(histogram); + } + + @Override + public OptionalLong distinctCount() { + return distinctCount; + } + + @Override + public Optional min() { + return min; + } + + @Override + public Optional max() { + return max; + } + + @Override + public OptionalLong nullCount() { + return nullCount; + } + + @Override + public OptionalLong avgLen() { + return avgLen; + } + + @Override + public OptionalLong maxLen() { + return maxLen; + } + + @Override + public Optional histogram() { + return histogram; + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 6efe8a080bde..8b88cf49c692 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -23,15 +23,19 @@ import java.util.Map; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.iceberg.BlobMetadata; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.StatisticsFile; import org.apache.iceberg.Table; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -75,22 +79,28 @@ import org.apache.iceberg.util.TableScanUtil; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.metric.CustomMetric; import org.apache.spark.sql.connector.metric.CustomTaskMetric; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsReportStatistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; abstract class SparkScan implements Scan, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; private final JavaSparkContext sparkContext; private final Table table; + private final SparkSession spark; private final SparkReadConf readConf; private final boolean caseSensitive; private final Schema expectedSchema; @@ -111,6 +121,7 @@ abstract class SparkScan implements Scan, SupportsReportStatistics { Schema snapshotSchema = SnapshotUtil.schemaFor(table, readConf.branch()); SparkSchemaUtil.validateMetadataColumnReferences(snapshotSchema, expectedSchema); + this.spark = spark; this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; this.readConf = readConf; @@ -175,7 +186,43 @@ public Statistics estimateStatistics() { protected Statistics estimateStatistics(Snapshot snapshot) { // its a fresh table, no data if (snapshot == null) { - return new Stats(0L, 0L); + return new Stats(0L, 0L, Collections.emptyMap()); + } + + boolean cboEnabled = + Boolean.parseBoolean(spark.conf().get(SQLConf.CBO_ENABLED().key(), "false")); + Map colStatsMap = Collections.emptyMap(); + if (readConf.reportColumnStats() && cboEnabled) { + colStatsMap = Maps.newHashMap(); + List files = table.statisticsFiles(); + if (!files.isEmpty()) { + List metadataList = (files.get(0)).blobMetadata(); + + for (BlobMetadata blobMetadata : metadataList) { + int id = blobMetadata.fields().get(0); + String colName = table.schema().findColumnName(id); + NamedReference ref = FieldReference.column(colName); + + Long ndv = null; + if (blobMetadata + .type() + .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { + String ndvStr = blobMetadata.properties().get(NDV_KEY); + if (!Strings.isNullOrEmpty(ndvStr)) { + ndv = Long.parseLong(ndvStr); + } else { + LOG.debug("ndv is not set in BlobMetadata for column {}", colName); + } + } else { + LOG.debug("DataSketch blob is not available for column {}", colName); + } + + ColumnStatistics colStats = + new SparkColumnStatistics(ndv, null, null, null, null, null, null); + + colStatsMap.put(ref, colStats); + } + } } // estimate stats using snapshot summary only for partitioned tables @@ -186,12 +233,13 @@ protected Statistics estimateStatistics(Snapshot snapshot) { snapshot.snapshotId(), table.name()); long totalRecords = totalRecords(snapshot); - return new Stats(SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords); + return new Stats( + SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords, colStatsMap); } long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); - return new Stats(sizeInBytes, rowsCount); + return new Stats(sizeInBytes, rowsCount, colStatsMap); } private long totalRecords(Snapshot snapshot) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java index ddf6ca834d9b..ccf523cb4b05 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java @@ -18,16 +18,21 @@ */ package org.apache.iceberg.spark.source; +import java.util.Map; import java.util.OptionalLong; +import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; class Stats implements Statistics { private final OptionalLong sizeInBytes; private final OptionalLong numRows; + private final Map colstats; - Stats(long sizeInBytes, long numRows) { + Stats(long sizeInBytes, long numRows, Map colstats) { this.sizeInBytes = OptionalLong.of(sizeInBytes); this.numRows = OptionalLong.of(numRows); + this.colstats = colstats; } @Override @@ -39,4 +44,9 @@ public OptionalLong sizeInBytes() { public OptionalLong numRows() { return numRows; } + + @Override + public Map columnStats() { + return colstats; + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index b67ee87c7d3e..601539784617 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TestSnapshot.testManifestStats; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; import static org.apache.spark.sql.functions.current_date; @@ -180,8 +181,10 @@ public void testBinPackUnpartitionedTable() { assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 1); - List actual = currentData(); + testManifestStats(table); + + List actual = currentData(); assertEquals("Rows must match", expectedRecords, actual); } @@ -200,8 +203,10 @@ public void testBinPackPartitionedTable() { assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); shouldHaveFiles(table, 4); - List actualRecords = currentData(); + testManifestStats(table); + + List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); } @@ -226,6 +231,8 @@ public void testBinPackWithFilter() { shouldHaveFiles(table, 7); + testManifestStats(table); + List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); } @@ -252,6 +259,8 @@ public void testBinPackWithFilterOnBucketExpression() { shouldHaveFiles(table, 7); + testManifestStats(table); + List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); } @@ -283,9 +292,10 @@ public void testBinPackAfterPartitionChange() { .hasSize(1); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + testManifestStats(table); + List postRewriteData = currentData(); assertEquals("We shouldn't have changed the data", originalData, postRewriteData); - shouldHaveSnapshots(table, 2); shouldHaveACleanCache(table); shouldHaveFiles(table, 20); @@ -331,6 +341,8 @@ public void testBinPackWithDeletes() { .isEqualTo(2); assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + testManifestStats(table); + List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); assertThat(actualRecords).as("7 rows are removed").hasSize(total - 7); @@ -361,6 +373,9 @@ public void testBinPackWithDeleteAllData() { .rewriteDataFiles(table) .option(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "1") .execute(); + + testManifestStats(table); + assertThat(result.rewrittenDataFilesCount()).as("Action should rewrite 1 data files").isOne(); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); @@ -435,6 +450,8 @@ public void testBinPackWithStartingSequenceNumberV1Compatibility() { assertThat(result.addedDataFilesCount()).as("Action should add 4 data files").isEqualTo(4); assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + testManifestStats(table); + shouldHaveFiles(table, 4); List actualRecords = currentData(); assertEquals("Rows must match", expectedRecords, actualRecords); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index d539b0123951..7d5475ff919e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import static org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createPartitionedTable; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createUnpartitionedTable; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToDayOrdinal; @@ -28,14 +29,22 @@ import static org.apache.spark.sql.functions.expr; import static org.assertj.core.api.Assertions.assertThat; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; import org.apache.iceberg.spark.functions.DaysFunction; @@ -44,6 +53,7 @@ import org.apache.iceberg.spark.functions.TruncateFunction; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; @@ -60,6 +70,8 @@ import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.junit.jupiter.api.AfterEach; @@ -130,6 +142,157 @@ public void testEstimatedRowCount() throws NoSuchTableException { assertThat(stats.numRows().getAsLong()).isEqualTo(10000L); } + @TestTemplate + public void testTableWithoutColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + + @TestTemplate + public void testTableWithOneColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + + @TestTemplate + public void testTableWithTwoColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(2), + ImmutableMap.of("ndv", "2")))); + + table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedTwoNDVs = Maps.newHashMap(); + expectedTwoNDVs.put("id", 4L); + expectedTwoNDVs.put("data", 2L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedTwoNDVs)); + } + @TestTemplate public void testUnpartitionedYears() throws Exception { createUnpartitionedTable(spark, tableName); @@ -734,6 +897,26 @@ private Expression[] expressions(Expression... expressions) { return expressions; } + private void checkColStatisticsNotReported(SparkScan scan, long expectedRowCount) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + assertThat(columnStats.isEmpty()); + } + + private void checkColStatisticsReported( + SparkScan scan, long expectedRowCount, Map expectedNDVs) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + for (Map.Entry entry : expectedNDVs.entrySet()) { + assertThat(columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } + } + private static LiteralValue intLit(int value) { return LiteralValue.apply(value, DataTypes.IntegerType); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSnapshotsTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSnapshotsTable.java new file mode 100644 index 000000000000..98c54e8cbb62 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSnapshotsTable.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.sql; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.spark.CatalogTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestSnapshotsTable extends CatalogTestBase { + + @BeforeEach + public void createTables() { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg " + + "TBLPROPERTIES" + + "('format-version'='2'," + + "'write.delete.mode'='merge-on-read')", + tableName); + sql("INSERT INTO %s VALUES (1, 'a1'),(2, 'a2'),(3, 'a3')", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testSnapshotsTable() { + List sql = sql("SELECT * FROM %s.%s", tableName, MetadataTableType.SNAPSHOTS); + assertThat(sql).hasSize(1); + } + + @TestTemplate + public void testTotalDataManifestFilesWithSnapshotsTableSummary() { + List sql = sql("SELECT * FROM %s.%s", tableName, MetadataTableType.SNAPSHOTS); + assertThat(sql).hasSize(1); + Map summary = (Map) sql.get(0)[5]; + assertThat(summary.get("total-data-manifest-files")).isEqualTo("1"); + assertThat(summary.get("total-delete-manifest-files")).isEqualTo(null); + sql("INSERT INTO %s VALUES (4, 'a4')", tableName); + sql = sql("SELECT * FROM %s.%s", tableName, MetadataTableType.SNAPSHOTS); + assertThat(sql).hasSize(2); + summary = (Map) sql.get(1)[5]; + assertThat(summary.get("total-data-manifest-files")).isEqualTo("2"); + assertThat(summary.get("total-delete-manifest-files")).isEqualTo(null); + } + + @TestTemplate + public void testTotalDeleteManifestFilesWithSnapshotsTableSummary() { + List sql = sql("SELECT * FROM %s.%s", tableName, MetadataTableType.SNAPSHOTS); + assertThat(sql).hasSize(1); + Map summary = (Map) sql.get(0)[5]; + assertThat(summary.get("total-data-manifest-files")).isEqualTo("1"); + assertThat(summary.get("total-delete-manifest-files")).isEqualTo(null); + sql("INSERT INTO %s VALUES (1, 'a1'),(2, 'a2'),(3, 'a3'),(4, 'a4')", tableName); + sql("INSERT INTO %s VALUES (1, 'a1'),(2, 'a2'),(3, 'a3'),(4, 'a4'),(5, 'a5')", tableName); + sql("INSERT INTO %s VALUES (1, 'b1'),(2, 'b2'),(3, 'b3'),(4, 'b4')", tableName); + sql("INSERT INTO %s VALUES (1, 'b1'),(2, 'b2'),(3, 'b3'),(4, 'b4'),(5, 'b5')", tableName); + sql = sql("SELECT * FROM %s.%s", tableName, MetadataTableType.SNAPSHOTS); + assertThat(sql).hasSize(5); + summary = (Map) sql.get(4)[5]; + assertThat(summary.get("total-data-manifest-files")).isEqualTo("5"); + assertThat(summary.get("total-delete-manifest-files")).isEqualTo(null); + + sql("DELETE FROM %s WHERE id = 1", tableName); + sql = sql("SELECT * FROM %s.%s", tableName, MetadataTableType.SNAPSHOTS); + assertThat(sql).hasSize(6); + summary = (Map) sql.get(5)[5]; + assertThat(summary.get("total-data-manifest-files")).isEqualTo("5"); + assertThat(summary.get("total-delete-manifest-files")).isEqualTo("1"); + } +} diff --git a/tasks.gradle b/tasks.gradle index 5fc24d41ae62..5515d7b75052 100644 --- a/tasks.gradle +++ b/tasks.gradle @@ -32,11 +32,9 @@ task aggregateJavadoc(type: Javadoc) { doLast { // Fix bug with search - if (JavaVersion.current() >= JavaVersion.VERSION_11) { - // Append the fix to the file - def searchScript = new File("site/docs/javadoc/${getJavadocVersion()}" + '/search.js') - searchScript.append JAVADOC_FIX_SEARCH_STR - } + // Append the fix to the file + def searchScript = new File("site/docs/javadoc/${getJavadocVersion()}" + '/search.js') + searchScript.append JAVADOC_FIX_SEARCH_STR } }