diff --git a/.asf.yaml b/.asf.yaml index 260641b35819..bf45b54a1787 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -49,12 +49,12 @@ github: - jun-he - marton-bod - samarthjain - - findepi - SreeramGarlapati - samredai - gaborkaszab - bitsondatadev - ajantha-bhat + - jbonofre ghp_branch: gh-pages ghp_path: / diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index c36700c8c122..570fb84d70a4 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -159,7 +159,6 @@ </module> <module name="ClassTypeParameterName"> <!-- Java Style Guide: Type variable names --> <property name="format" value="(^[A-Z][0-9]?)$|([A-Z][a-zA-Z0-9]*[T]$)"/> - <message key="name.invalidPattern" value="Class type name ''{0}'' must match pattern ''{1}''."/> </module> <module name="CovariantEquals"/> <!-- Java Coding Guidelines: Override ``Object#equals`` consistently --> <module name="DefaultComesLast"/> <!-- Java Style Guide: The default case is present --> @@ -281,12 +280,13 @@ </module> <module name="InnerAssignment"/> <!-- Java Coding Guidelines: Inner assignments: Not used --> <module name="MemberName"> <!-- Java Style Guide: Non-constant field names --> - <property name="format" value="^[a-z][a-zA-Z0-9]+$"/> - <message key="name.invalidPattern" value="Member name ''{0}'' must match pattern ''{1}''."/> + <property name="format" value="^[a-z][a-zA-Z0-9]++$"/> + </module> + <module name="ConstantName"> + <property name="format" value="^[A-Z][A-Z0-9]*+(_[A-Z0-9]++)*+$"/> </module> <module name="MethodName"> <!-- Java Style Guide: Method names --> - <property name="format" value="^[a-z][a-zA-Z0-9_]+$"/> - <message key="name.invalidPattern" value="Method name ''{0}'' must match pattern ''{1}''."/> + <property name="format" value="^[a-z][a-zA-Z0-9_]++$"/> </module> <module name="MethodParamPad"/> <!-- Java Style Guide: Horizontal whitespace --> <module name="MissingDeprecated"/> <!-- Java Coding Guide: Deprecate per annotation and Javadoc --> @@ -304,8 +304,7 @@ <module name="PackageAnnotation"/> <!-- Java Style Guide: Package statement --> <module name="PackageDeclaration"/> <!-- Java Style Guide: Package statement --> <module name="PackageName"> <!-- Java Style Guide: Package names --> - <property name="format" value="^[a-z]+(\.[a-z][a-z0-9]*)*$"/> - <message key="name.invalidPattern" value="Package name ''{0}'' must match pattern ''{1}''."/> + <property name="format" value="^[a-z]++(\.[a-z][a-z0-9]*+)*+$"/> </module> <module name="ParameterAssignment"/> <!-- Java Coding Guidelines: Final variables and parameters --> <module name="ParenPad"/> <!-- Java Style Guide: Horizontal whitespace --> @@ -415,6 +414,9 @@ <property name="format" value="@Test\(.*expected.*\)"/> <property name="message" value="Prefer using Assertions.assertThatThrownBy(...).isInstanceOf(...) instead."/> </module> + <module name="IllegalToken"> + <property name="tokens" value="LITERAL_ASSERT"/> + </module> <module name="IllegalImport"> <property name="id" value="BanExpectedExceptionUsage"/> <property name="illegalClasses" value="org.junit.rules.ExpectedException"/> @@ -444,7 +446,6 @@ <message key="todo.match" value="There must be whitespace at the beginning of all comments."/> </module> <module name="TypeName"> <!-- Java Style Guide: Class names --> - <message key="name.invalidPattern" value="Type name ''{0}'' must match pattern ''{1}''."/> </module> <module name="TypecastParenPad"/> <!-- Java Style Guide: Horizontal whitespace --> <module name="UnusedImports"> <!-- Java Style Guide: No unused imports --> @@ -480,14 +481,12 @@ <module name="LocalFinalVariableName"/> <!-- Java Style Guide: Local variable names --> <module name="LocalVariableName"> <!-- Java Style Guide: Local variable names --> <property name="tokens" value="VARIABLE_DEF"/> - <property name="format" value="^[a-z][a-zA-Z0-9]+$"/> + <property name="format" value="^[a-z][a-zA-Z0-9]++$"/> <property name="allowOneCharVarInForLoop" value="true"/> - <message key="name.invalidPattern" value="Local variable name ''{0}'' must match pattern ''{1}''."/> </module> <module name="MethodLength"/> <!-- Java Coding Guide: Methods and functions: focused, crisp, concise --> <module name="MethodTypeParameterName"> <!-- Java Style Guide: Type variable names --> <property name="format" value="(^[A-Z][0-9]?)$|([A-Z][a-zA-Z0-9]*[T]$)"/> - <message key="name.invalidPattern" value="Method type name ''{0}'' must match pattern ''{1}''."/> </module> <module name="NestedForDepth"> <property name="max" value="2"/> @@ -495,8 +494,7 @@ <module name="NestedTryDepth"/> <!-- Java Coding Guide: Try/catch blocks: never nested --> <module name="NonEmptyAtclauseDescription"/> <!-- Java Style Guide: At-clauses --> <module name="ParameterName"> <!-- Java Style Guide: Parameter names --> - <property name="format" value="^[a-z][a-zA-Z0-9]+$"/> - <message key="name.invalidPattern" value="Parameter name ''{0}'' must match pattern ''{1}''."/> + <property name="format" value="^[a-z][a-zA-Z0-9]++$"/> <property name="ignoreOverridden" value="true"/> </module> diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index a6d4fc7e5c2e..a04263e8c943 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -28,7 +28,9 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.5.2 (latest release)" + - "1.6.1 (latest release)" + - "1.6.0" + - "1.5.2" - "1.5.1" - "1.5.0" - "1.4.3" @@ -76,3 +78,11 @@ body: You can include files by dragging and dropping them here. validations: required: true + - type: checkboxes + attributes: + label: Willingness to contribute + description: The Apache Iceberg community encourages bug fix contributions. Would you or another member of your organization be willing to contribute a fix for this bug to the Apache Iceberg codebase? + options: + - label: I can contribute a fix for this bug independently + - label: I would be willing to contribute a fix for this bug with guidance from the Iceberg community + - label: I cannot contribute a fix for this bug at this time 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 90c1ad4109fd..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] + 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] + 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 b74cbcc84291..370375783cc2 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -73,12 +73,8 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] - flink: ['1.17', '1.18', '1.19'] - exclude: - # Flink 1.17 does not support Java 17. - - jvm: 17 - flink: '1.17' + 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 6e97e2164758..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] + 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] + 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 083ae2f5fe36..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] + 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] + 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] + 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 d923cf860f1d..1cc0425b73a3 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -71,9 +71,16 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [11, 17, 21] spark: ['3.3', '3.4', '3.5'] scala: ['2.12', '2.13'] + exclude: + # Spark 3.5 is the first version not failing on Java 21 (https://issues.apache.org/jira/browse/SPARK-42369) + # Full Java 21 support is coming in Spark 4 (https://issues.apache.org/jira/browse/SPARK-43831) + - jvm: 21 + spark: '3.3' + - jvm: 21 + spark: '3.4' env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 0f599e5582cc..03276d60b524 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -25,6 +25,7 @@ on: permissions: # All other permissions are set to none issues: write + pull-requests: write jobs: stale: @@ -54,4 +55,4 @@ jobs: days-before-pr-stale: 30 days-before-pr-close: 7 ascending: true - operations-per-run: 100 + operations-per-run: 200 diff --git a/.gitignore b/.gitignore index 6740a061e346..e89662a10c6d 100644 --- a/.gitignore +++ b/.gitignore @@ -37,7 +37,7 @@ spark/v3.4/spark/benchmark/* spark/v3.4/spark-extensions/benchmark/* spark/v3.5/spark/benchmark/* spark/v3.5/spark-extensions/benchmark/* -data/benchmark/* +*/benchmark/* __pycache__/ *.py[cod] diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 3018840b4513..9b8017f0beec 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1025,14 +1025,11 @@ acceptedBreaks: new: "class org.apache.iceberg.types.Types.NestedField" justification: "new Constructor added" org.apache.iceberg:iceberg-core: - - code: "java.method.visibilityReduced" - old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()" - new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()" - justification: "Deprecations for 1.6.0 release" - code: "java.element.noLongerDeprecated" old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()" new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()" - justification: "Constructor became private as part of deprecations cleanup for 1.6.0 release" + justification: "Constructor became private as part of deprecations cleanup for\ + \ 1.6.0 release" - code: "java.element.noLongerDeprecated" old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::<init>(java.util.Map<java.lang.String,\ \ java.lang.String>, java.lang.String, java.lang.String, java.lang.String,\ @@ -1056,6 +1053,89 @@ acceptedBreaks: - code: "java.method.removed" old: "method org.apache.iceberg.DataFiles.Builder org.apache.iceberg.DataFiles.Builder::withEqualityFieldIds(java.util.List<java.lang.Integer>)" justification: "Deprecations for 1.6.0 release" + - code: "java.method.visibilityReduced" + old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()" + new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::<init>()" + justification: "Deprecations for 1.6.0 release" + "1.6.0": + org.apache.iceberg:iceberg-common: + - code: "java.method.removed" + old: "method <T> org.apache.iceberg.common.DynFields.StaticField<T> org.apache.iceberg.common.DynFields.Builder::buildStaticChecked()\ + \ throws java.lang.NoSuchFieldException" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method java.lang.Class<? extends C> org.apache.iceberg.common.DynConstructors.Ctor<C>::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> R org.apache.iceberg.common.DynMethods.UnboundMethod::invokeChecked(java.lang.Object,\ + \ java.lang.Object[]) throws java.lang.Exception" + new: "method <R> 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.defaultSerializationChanged" + old: "class org.apache.iceberg.GenericManifestFile" + new: "class org.apache.iceberg.GenericManifestFile" + justification: "Serialization across versions is not supported" + - code: "java.class.removed" + old: "enum org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus" + justification: "Removing deprecated code" + - 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<java.lang.String, org.apache.iceberg.io.ContentCache.FileContent>)" + 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<ThisT>::newManifestOutput()\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer<ThisT>::newManifestOutput()\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer<ThisT>::newManifestOutput()\ + \ @ org.apache.iceberg.BaseRewriteManifests" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer<ThisT>::newManifestOutput()\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::<init>(java.util.Map<java.lang.String,\ + \ java.lang.String>, 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 8200a23ed63b..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, or 17. +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/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java index b161bfcaf767..64e625997760 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java @@ -157,7 +157,7 @@ private void closeStream() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java index 9644dab82285..8c226d00123c 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java @@ -165,7 +165,7 @@ private void cleanUpStagingFiles() { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java index 069ff9111a5a..8fc661e5be10 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java @@ -50,7 +50,7 @@ public class TestOSSOutputStream extends AliyunOSSTestBase { private final OSS ossMock = mock(OSS.class, delegatesTo(ossClient)); private final Path tmpDir = Files.createTempDirectory("oss-file-io-test-"); - private static final Random random = ThreadLocalRandom.current(); + private static final Random RANDOM = ThreadLocalRandom.current(); private final AliyunProperties props = new AliyunProperties( @@ -127,7 +127,7 @@ private byte[] data256() { private byte[] randomData(int size) { byte[] data = new byte[size]; - random.nextBytes(data); + RANDOM.nextBytes(data); return data; } diff --git a/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<String, String> 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/Metrics.java b/api/src/main/java/org/apache/iceberg/Metrics.java index 2f2cf89cdadd..d1a7da6d9d95 100644 --- a/api/src/main/java/org/apache/iceberg/Metrics.java +++ b/api/src/main/java/org/apache/iceberg/Metrics.java @@ -179,6 +179,7 @@ private static void writeByteBufferMap( * @throws IOException On serialization error * @throws ClassNotFoundException If the class is not found */ + @SuppressWarnings("DangerousJavaDeserialization") private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { rowCount = (Long) in.readObject(); columnSizes = (Map<Integer, Long>) in.readObject(); @@ -190,6 +191,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE upperBounds = readByteBufferMap(in); } + @SuppressWarnings("DangerousJavaDeserialization") private static Map<Integer, ByteBuffer> readByteBufferMap(ObjectInputStream in) throws IOException, ClassNotFoundException { int size = in.readInt(); diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 08a1c4f9ecfd..9b74893f1831 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -371,6 +371,7 @@ public static class Builder { new AtomicInteger(unpartitionedLastAssignedId()); // check if there are conflicts between partition and schema field name private boolean checkConflicts = true; + private boolean caseSensitive = true; private Builder(Schema schema) { this.schema = schema; @@ -390,7 +391,8 @@ Builder checkConflicts(boolean check) { } private void checkAndAddPartitionName(String name, Integer sourceColumnId) { - Types.NestedField schemaField = schema.findField(name); + Types.NestedField schemaField = + this.caseSensitive ? schema.findField(name) : schema.caseInsensitiveFindField(name); if (checkConflicts) { if (sourceColumnId != null) { // for identity transform case we allow conflicts between partition and schema field name @@ -427,20 +429,31 @@ private void checkForRedundantPartitions(PartitionField field) { dedupFields.put(dedupKey, field); } + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + public Builder withSpecId(int newSpecId) { this.specId = newSpecId; return this; } private Types.NestedField findSourceColumn(String sourceName) { - Types.NestedField sourceColumn = schema.findField(sourceName); + Types.NestedField sourceColumn = + this.caseSensitive + ? schema.findField(sourceName) + : schema.caseInsensitiveFindField(sourceName); Preconditions.checkArgument( sourceColumn != null, "Cannot find source column: %s", sourceName); return sourceColumn; } Builder identity(String sourceName, String targetName) { - Types.NestedField sourceColumn = findSourceColumn(sourceName); + return identity(findSourceColumn(sourceName), targetName); + } + + private Builder identity(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName, sourceColumn.fieldId()); PartitionField field = new PartitionField( @@ -451,12 +464,16 @@ Builder identity(String sourceName, String targetName) { } public Builder identity(String sourceName) { - return identity(sourceName, sourceName); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + return identity(sourceColumn, schema.findColumnName(sourceColumn.fieldId())); } public Builder year(String sourceName, String targetName) { + return year(findSourceColumn(sourceName), targetName); + } + + private Builder year(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.year()); checkForRedundantPartitions(field); @@ -465,12 +482,17 @@ public Builder year(String sourceName, String targetName) { } public Builder year(String sourceName) { - return year(sourceName, sourceName + "_year"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return year(sourceColumn, columnName + "_year"); } public Builder month(String sourceName, String targetName) { + return month(findSourceColumn(sourceName), targetName); + } + + private Builder month(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.month()); checkForRedundantPartitions(field); @@ -479,12 +501,17 @@ public Builder month(String sourceName, String targetName) { } public Builder month(String sourceName) { - return month(sourceName, sourceName + "_month"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return month(sourceColumn, columnName + "_month"); } public Builder day(String sourceName, String targetName) { + return day(findSourceColumn(sourceName), targetName); + } + + private Builder day(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.day()); checkForRedundantPartitions(field); @@ -493,12 +520,17 @@ public Builder day(String sourceName, String targetName) { } public Builder day(String sourceName) { - return day(sourceName, sourceName + "_day"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return day(sourceColumn, columnName + "_day"); } public Builder hour(String sourceName, String targetName) { + return hour(findSourceColumn(sourceName), targetName); + } + + private Builder hour(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.hour()); checkForRedundantPartitions(field); @@ -507,12 +539,17 @@ public Builder hour(String sourceName, String targetName) { } public Builder hour(String sourceName) { - return hour(sourceName, sourceName + "_hour"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return hour(sourceColumn, columnName + "_hour"); } public Builder bucket(String sourceName, int numBuckets, String targetName) { + return bucket(findSourceColumn(sourceName), numBuckets, targetName); + } + + private Builder bucket(Types.NestedField sourceColumn, int numBuckets, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); fields.add( new PartitionField( sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.bucket(numBuckets))); @@ -520,12 +557,17 @@ public Builder bucket(String sourceName, int numBuckets, String targetName) { } public Builder bucket(String sourceName, int numBuckets) { - return bucket(sourceName, numBuckets, sourceName + "_bucket"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return bucket(sourceColumn, numBuckets, columnName + "_bucket"); } public Builder truncate(String sourceName, int width, String targetName) { + return truncate(findSourceColumn(sourceName), width, targetName); + } + + private Builder truncate(Types.NestedField sourceColumn, int width, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); fields.add( new PartitionField( sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.truncate(width))); @@ -533,11 +575,16 @@ public Builder truncate(String sourceName, int width, String targetName) { } public Builder truncate(String sourceName, int width) { - return truncate(sourceName, width, sourceName + "_trunc"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return truncate(sourceColumn, width, columnName + "_trunc"); } public Builder alwaysNull(String sourceName, String targetName) { - Types.NestedField sourceColumn = findSourceColumn(sourceName); + return alwaysNull(findSourceColumn(sourceName), targetName); + } + + private Builder alwaysNull(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName( targetName, sourceColumn.fieldId()); // can duplicate a source column name fields.add( @@ -547,7 +594,9 @@ public Builder alwaysNull(String sourceName, String targetName) { } public Builder alwaysNull(String sourceName) { - return alwaysNull(sourceName, sourceName + "_null"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return alwaysNull(sourceColumn, columnName + "_null"); } // add a partition field with an auto-increment partition field id starting from diff --git a/api/src/main/java/org/apache/iceberg/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/RewriteManifests.java index ca823e94d265..32a9011ad6f6 100644 --- a/api/src/main/java/org/apache/iceberg/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/RewriteManifests.java @@ -54,7 +54,7 @@ public interface RewriteManifests extends SnapshotUpdate<RewriteManifests> { * then all manifests will be rewritten. * * @param predicate Predicate used to determine which manifests to rewrite. If true then the - * manifest file will be included for rewrite. If false then then manifest is kept as-is. + * manifest file will be included for rewrite. If false then the manifest is kept as-is. * @return this for method chaining */ RewriteManifests rewriteIf(Predicate<ManifestFile> predicate); diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 7ff712b62790..9bcf691f5a03 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -54,6 +54,8 @@ public class Schema implements Serializable { private static final Joiner NEWLINE = Joiner.on('\n'); private static final String ALL_COLUMNS = "*"; private static final int DEFAULT_SCHEMA_ID = 0; + private static final Map<Type.TypeID, Integer> MIN_FORMAT_VERSIONS = + ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3); private final StructType struct; private final int schemaId; @@ -573,4 +575,27 @@ private List<NestedField> reassignIds(List<NestedField> columns, TypeUtil.GetID }); return res.asStructType().fields(); } + + /** + * Check the compatibility of the schema with a format version. + * + * <p>This validates that the schema does not contain types that were released in later format + * versions. + * + * @param schema a Schema + * @param formatVersion table format version + */ + public static void checkCompatibility(Schema schema, int formatVersion) { + // check the type in each field + for (NestedField field : schema.lazyIdToField().values()) { + Integer minFormatVersion = MIN_FORMAT_VERSIONS.get(field.type().typeId()); + Preconditions.checkState( + minFormatVersion == null || formatVersion >= minFormatVersion, + "Invalid type in v%s schema: %s %s is not supported until v%s", + formatVersion, + schema.findColumnName(field.fieldId()), + field.type(), + minFormatVersion); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/StatisticsFile.java b/api/src/main/java/org/apache/iceberg/StatisticsFile.java index f3d74b72852a..8f5166cf9fad 100644 --- a/api/src/main/java/org/apache/iceberg/StatisticsFile.java +++ b/api/src/main/java/org/apache/iceberg/StatisticsFile.java @@ -28,7 +28,7 @@ * support is not required to read the table correctly. */ public interface StatisticsFile { - /** ID of the Iceberg table's snapshot the statistics were computed from. */ + /** ID of the Iceberg table's snapshot the statistics file is associated with. */ long snapshotId(); /** diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index 6da9bbf2f4fe..97ea9ba76526 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -270,7 +270,7 @@ default AppendFiles newFastAppend() { ReplacePartitions newReplacePartitions(); /** - * Create a new {@link DeleteFiles delete API} to replace files in this table and commit. + * Create a new {@link DeleteFiles delete API} to delete files in this table and commit. * * @return a new {@link DeleteFiles} */ @@ -299,7 +299,7 @@ default UpdatePartitionStatistics updatePartitionStatistics() { } /** - * Create a new {@link ExpireSnapshots expire API} to manage snapshots in this table and commit. + * Create a new {@link ExpireSnapshots expire API} to expire snapshots in this table and commit. * * @return a new {@link ExpireSnapshots} */ diff --git a/api/src/main/java/org/apache/iceberg/Transaction.java b/api/src/main/java/org/apache/iceberg/Transaction.java index fd84a974013d..77e19e45e451 100644 --- a/api/src/main/java/org/apache/iceberg/Transaction.java +++ b/api/src/main/java/org/apache/iceberg/Transaction.java @@ -52,7 +52,7 @@ public interface Transaction { UpdateProperties updateProperties(); /** - * Create a new {@link ReplaceSortOrder} to set a table sort order and commit the change. + * Create a new {@link ReplaceSortOrder} to set a table sort order. * * @return a new {@link ReplaceSortOrder} */ @@ -131,7 +131,7 @@ default AppendFiles newFastAppend() { ReplacePartitions newReplacePartitions(); /** - * Create a new {@link DeleteFiles delete API} to replace files in this table. + * Create a new {@link DeleteFiles delete API} to delete files in this table. * * @return a new {@link DeleteFiles} */ @@ -160,7 +160,7 @@ default UpdatePartitionStatistics updatePartitionStatistics() { } /** - * Create a new {@link ExpireSnapshots expire API} to manage snapshots in this table. + * Create a new {@link ExpireSnapshots expire API} to expire snapshots in this table. * * @return a new {@link ExpireSnapshots} */ 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<PartitionSpec> { * 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/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index 2d6ff2679a17..85773febae17 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -70,4 +70,10 @@ default RewritePositionDeleteFiles rewritePositionDeletes(Table table) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement rewritePositionDeletes"); } + + /** Instantiates an action to compute table stats. */ + default ComputeTableStats computeTableStats(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement computeTableStats"); + } } diff --git a/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java b/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java new file mode 100644 index 000000000000..04449d591657 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import org.apache.iceberg.StatisticsFile; + +/** An action that collects statistics of an Iceberg table and writes to Puffin files. */ +public interface ComputeTableStats extends Action<ComputeTableStats, ComputeTableStats.Result> { + /** + * Choose the set of columns to collect stats, by default all columns are chosen. + * + * @param columns a set of column names to be analyzed + * @return this for method chaining + */ + ComputeTableStats columns(String... columns); + + /** + * Choose the table snapshot to compute stats, by default the current snapshot is used. + * + * @param snapshotId long ID of the snapshot for which stats need to be computed + * @return this for method chaining + */ + ComputeTableStats snapshot(long snapshotId); + + /** The result of table statistics collection. */ + interface Result { + + /** Returns statistics file or none if no statistics were collected. */ + StatisticsFile statisticsFile(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/events/Listeners.java b/api/src/main/java/org/apache/iceberg/events/Listeners.java index 27c9c0590658..429cf7c7e272 100644 --- a/api/src/main/java/org/apache/iceberg/events/Listeners.java +++ b/api/src/main/java/org/apache/iceberg/events/Listeners.java @@ -28,11 +28,11 @@ public class Listeners { private Listeners() {} - private static final Map<Class<?>, Queue<Listener<?>>> listeners = Maps.newConcurrentMap(); + private static final Map<Class<?>, Queue<Listener<?>>> LISTENERS = Maps.newConcurrentMap(); public static <E> void register(Listener<E> listener, Class<E> eventType) { Queue<Listener<?>> list = - listeners.computeIfAbsent(eventType, k -> new ConcurrentLinkedQueue<>()); + LISTENERS.computeIfAbsent(eventType, k -> new ConcurrentLinkedQueue<>()); list.add(listener); } @@ -40,7 +40,7 @@ public static <E> void register(Listener<E> listener, Class<E> eventType) { public static <E> void notifyAll(E event) { Preconditions.checkNotNull(event, "Cannot notify listeners for a null event."); - Queue<Listener<?>> list = listeners.get(event.getClass()); + Queue<Listener<?>> list = LISTENERS.get(event.getClass()); if (list != null) { for (Listener<?> value : list) { Listener<E> listener = (Listener<E>) value; diff --git a/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java b/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java index 02dc31c6a6c5..127d46e6a48f 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java @@ -31,6 +31,7 @@ public class BoundLiteralPredicate<T> extends BoundPredicate<T> { Type.TypeID.LONG, Type.TypeID.DATE, Type.TypeID.TIME, + Type.TypeID.TIMESTAMP_NANO, Type.TypeID.TIMESTAMP); private static long toLong(Literal<?> lit) { diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java index cb40821733a3..8b9986a260e8 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -36,6 +36,7 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; /** Expression utility methods. */ public class ExpressionUtil { @@ -43,6 +44,7 @@ public class ExpressionUtil { Transforms.bucket(Integer.MAX_VALUE).bind(Types.StringType.get()); private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); private static final long FIVE_MINUTES_IN_MICROS = TimeUnit.MINUTES.toMicros(5); + private static final long FIVE_MINUTES_IN_NANOS = TimeUnit.MINUTES.toNanos(5); private static final long THREE_DAYS_IN_HOURS = TimeUnit.DAYS.toHours(3); private static final long NINETY_DAYS_IN_HOURS = TimeUnit.DAYS.toHours(90); private static final Pattern DATE = Pattern.compile("\\d{4}-\\d{2}-\\d{2}"); @@ -52,6 +54,12 @@ public class ExpressionUtil { private static final Pattern TIMESTAMPTZ = Pattern.compile( "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,9})?)?([-+]\\d{2}:\\d{2}|Z)"); + private static final Pattern TIMESTAMPNS = + Pattern.compile("\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?"); + private static final Pattern TIMESTAMPTZNS = + Pattern.compile( + "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?([-+]\\d{2}:\\d{2}|Z)"); + static final int LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD = 10; private static final int LONG_IN_PREDICATE_ABBREVIATION_MIN_GAIN = 5; @@ -545,6 +553,8 @@ private static String sanitize(Type type, Object value, long now, int today) { return "(time)"; case TIMESTAMP: return sanitizeTimestamp((long) value, now); + case TIMESTAMP_NANO: + return sanitizeTimestamp(DateTimeUtil.nanosToMicros((long) value / 1000), now); case STRING: return sanitizeString((CharSequence) value, now, today); case BOOLEAN: @@ -566,6 +576,9 @@ private static String sanitize(Literal<?> literal, long now, int today) { return sanitizeDate(((Literals.DateLiteral) literal).value(), today); } else if (literal instanceof Literals.TimestampLiteral) { return sanitizeTimestamp(((Literals.TimestampLiteral) literal).value(), now); + } else if (literal instanceof Literals.TimestampNanoLiteral) { + return sanitizeTimestamp( + DateTimeUtil.nanosToMicros(((Literals.TimestampNanoLiteral) literal).value()), now); } else if (literal instanceof Literals.TimeLiteral) { return "(time)"; } else if (literal instanceof Literals.IntegerLiteral) { @@ -624,6 +637,12 @@ private static String sanitizeString(CharSequence value, long now, int today) { if (DATE.matcher(value).matches()) { Literal<Integer> date = Literal.of(value).to(Types.DateType.get()); return sanitizeDate(date.value(), today); + } else if (TIMESTAMPNS.matcher(value).matches()) { + Literal<Long> ts = Literal.of(value).to(Types.TimestampNanoType.withoutZone()); + return sanitizeTimestamp(DateTimeUtil.nanosToMicros(ts.value()), now); + } else if (TIMESTAMPTZNS.matcher(value).matches()) { + Literal<Long> ts = Literal.of(value).to(Types.TimestampNanoType.withZone()); + return sanitizeTimestamp(DateTimeUtil.nanosToMicros(ts.value()), now); } else if (TIMESTAMP.matcher(value).matches()) { Literal<Long> ts = Literal.of(value).to(Types.TimestampType.withoutZone()); return sanitizeTimestamp(ts.value(), now); diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 79d7190c49df..ee47035b1e72 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -24,7 +24,6 @@ import java.nio.ByteBuffer; import java.time.Instant; import java.time.LocalDate; -import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -40,6 +39,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.NaNUtil; class Literals { @@ -299,6 +299,9 @@ public <T> Literal<T> to(Type type) { return (Literal<T>) new TimeLiteral(value()); case TIMESTAMP: return (Literal<T>) new TimestampLiteral(value()); + case TIMESTAMP_NANO: + // assume micros and convert to nanos to match the behavior in the timestamp case above + return new TimestampLiteral(value()).to(type); case DATE: if ((long) Integer.MAX_VALUE < value()) { return aboveMax(); @@ -437,11 +440,9 @@ public <T> Literal<T> to(Type type) { case TIMESTAMP: return (Literal<T>) this; case DATE: - return (Literal<T>) - new DateLiteral( - (int) - ChronoUnit.DAYS.between( - EPOCH_DAY, EPOCH.plus(value(), ChronoUnit.MICROS).toLocalDate())); + return (Literal<T>) new DateLiteral(DateTimeUtil.microsToDays(value())); + case TIMESTAMP_NANO: + return (Literal<T>) new TimestampNanoLiteral(DateTimeUtil.microsToNanos(value())); default: } return null; @@ -453,6 +454,32 @@ protected Type.TypeID typeId() { } } + static class TimestampNanoLiteral extends ComparableLiteral<Long> { + TimestampNanoLiteral(Long value) { + super(value); + } + + @Override + @SuppressWarnings("unchecked") + public <T> Literal<T> to(Type type) { + switch (type.typeId()) { + case DATE: + return (Literal<T>) new DateLiteral(DateTimeUtil.nanosToDays(value())); + case TIMESTAMP: + return (Literal<T>) new TimestampLiteral(DateTimeUtil.nanosToMicros(value())); + case TIMESTAMP_NANO: + return (Literal<T>) this; + default: + } + return null; + } + + @Override + protected Type.TypeID typeId() { + return Type.TypeID.TIMESTAMP_NANO; + } + } + static class DecimalLiteral extends ComparableLiteral<BigDecimal> { DecimalLiteral(BigDecimal value) { super(value); @@ -502,19 +529,21 @@ public <T> Literal<T> to(Type type) { case TIMESTAMP: if (((Types.TimestampType) type).shouldAdjustToUTC()) { - long timestampMicros = - ChronoUnit.MICROS.between( - EPOCH, OffsetDateTime.parse(value(), DateTimeFormatter.ISO_DATE_TIME)); + long timestampMicros = DateTimeUtil.isoTimestamptzToMicros(value().toString()); return (Literal<T>) new TimestampLiteral(timestampMicros); } else { - long timestampMicros = - ChronoUnit.MICROS.between( - EPOCH, - LocalDateTime.parse(value(), DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .atOffset(ZoneOffset.UTC)); + long timestampMicros = DateTimeUtil.isoTimestampToMicros(value().toString()); return (Literal<T>) new TimestampLiteral(timestampMicros); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return (Literal<T>) + new TimestampNanoLiteral(DateTimeUtil.isoTimestamptzToNanos(value())); + } else { + return (Literal<T>) new TimestampNanoLiteral(DateTimeUtil.isoTimestampToNanos(value())); + } + case STRING: return (Literal<T>) this; 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/transforms/Bucket.java b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java index 912bcd271725..0e4e782cc110 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java @@ -33,6 +33,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.BucketUtil; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.SerializableFunction; class Bucket<T> implements Transform<T, Integer>, Serializable { @@ -63,6 +64,8 @@ static <T, B extends Bucket<T> & SerializableFunction<T, Integer>> B get( case FIXED: case BINARY: return (B) new BucketByteBuffer(numBuckets); + case TIMESTAMP_NANO: + return (B) new BucketTimestampNano(numBuckets); case UUID: return (B) new BucketUUID(numBuckets); default: @@ -107,6 +110,7 @@ public boolean canTransform(Type type) { case DATE: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: case STRING: case BINARY: case FIXED: @@ -214,6 +218,20 @@ protected int hash(Long value) { } } + // In order to bucket TimestampNano the same as Timestamp, convert to micros before hashing. + private static class BucketTimestampNano extends Bucket<Long> + implements SerializableFunction<Long, Integer> { + + private BucketTimestampNano(int numBuckets) { + super(numBuckets); + } + + @Override + protected int hash(Long nanos) { + return BucketUtil.hash(DateTimeUtil.nanosToMicros(nanos)); + } + } + private static class BucketString extends Bucket<CharSequence> implements SerializableFunction<CharSequence, Integer> { diff --git a/api/src/main/java/org/apache/iceberg/transforms/Dates.java b/api/src/main/java/org/apache/iceberg/transforms/Dates.java index 3d26b542be7b..88db16797867 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Dates.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Dates.java @@ -97,6 +97,10 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } + ChronoUnit granularity() { + return granularity; + } + @Override public boolean preservesOrder() { return true; @@ -109,11 +113,11 @@ public boolean satisfiesOrderOf(Transform<?, ?> other) { } if (other instanceof Dates) { - // test the granularity, in days. day(ts) => 1 day, months(ts) => 30 days, and day satisfies - // the order of months - Dates otherTransform = (Dates) other; - return granularity.getDuration().toDays() - <= otherTransform.granularity.getDuration().toDays(); + return TransformUtil.satisfiesOrderOf(granularity, ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity, ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf(granularity, ((TimeTransform<?>) other).granularity()); } return false; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Days.java b/api/src/main/java/org/apache/iceberg/transforms/Days.java index f69d5d6110ed..e2b829b86662 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Days.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Days.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static <T> Days<T> get() { } @Override - @SuppressWarnings("unchecked") - protected Transform<T, Integer> toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform<T, Integer>) Dates.DAY; - case TIMESTAMP: - return (Transform<T, Integer>) Timestamps.DAY; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.DAYS; } @Override - public Type getResultType(Type sourceType) { - return Types.DateType.get(); + protected Transform<T, Integer> toEnum(Type type) { + return (Transform<T, Integer>) + fromSourceType(type, Dates.DAY, Timestamps.MICROS_TO_DAY, Timestamps.NANOS_TO_DAY); } @Override - public boolean satisfiesOrderOf(Transform<?, ?> other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.DAY.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.DAY.satisfiesOrderOf(other); - } else if (other instanceof Days || other instanceof Months || other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.DateType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/Hours.java b/api/src/main/java/org/apache/iceberg/transforms/Hours.java index afc14516f3cd..2ff79f6a66a7 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Hours.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Hours.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -30,19 +31,21 @@ static <T> Hours<T> get() { return (Hours<T>) INSTANCE; } + @Override + protected ChronoUnit granularity() { + return ChronoUnit.HOURS; + } + @Override @SuppressWarnings("unchecked") protected Transform<T, Integer> toEnum(Type type) { - if (type.typeId() == Type.TypeID.TIMESTAMP) { - return (Transform<T, Integer>) Timestamps.HOUR; - } - - throw new IllegalArgumentException("Unsupported type: " + type); + return (Transform<T, Integer>) + fromSourceType(type, null, Timestamps.MICROS_TO_HOUR, Timestamps.NANOS_TO_HOUR); } @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.TIMESTAMP || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override @@ -50,24 +53,6 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } - @Override - public boolean satisfiesOrderOf(Transform<?, ?> other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return other == Timestamps.HOUR; - } else if (other instanceof Hours - || other instanceof Days - || other instanceof Months - || other instanceof Years) { - return true; - } - - return false; - } - @Override public String toHumanString(Type alwaysInt, Integer value) { return value != null ? TransformUtil.humanHour(value) : "null"; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Months.java b/api/src/main/java/org/apache/iceberg/transforms/Months.java index 8fa4d42385f7..73ec50e5dd9a 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Months.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Months.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static <T> Months<T> get() { } @Override - @SuppressWarnings("unchecked") - protected Transform<T, Integer> toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform<T, Integer>) Dates.MONTH; - case TIMESTAMP: - return (Transform<T, Integer>) Timestamps.MONTH; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.MONTHS; } @Override - public Type getResultType(Type sourceType) { - return Types.IntegerType.get(); + protected Transform<T, Integer> toEnum(Type type) { + return (Transform<T, Integer>) + fromSourceType(type, Dates.MONTH, Timestamps.MICROS_TO_MONTH, Timestamps.NANOS_TO_MONTH); } @Override - public boolean satisfiesOrderOf(Transform<?, ?> other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.MONTH.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.MONTH.satisfiesOrderOf(other); - } else if (other instanceof Months || other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.IntegerType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java index e4796478bf28..0d80ef88a296 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java @@ -122,16 +122,23 @@ static <R> R visit(Schema schema, PartitionField field, PartitionSpecVisitor<R> int width = ((Truncate<?>) transform).width(); return visitor.truncate(field.fieldId(), sourceName, field.sourceId(), width); } else if (transform == Dates.YEAR - || transform == Timestamps.YEAR + || transform == Timestamps.MICROS_TO_YEAR + || transform == Timestamps.NANOS_TO_YEAR || transform instanceof Years) { return visitor.year(field.fieldId(), sourceName, field.sourceId()); } else if (transform == Dates.MONTH - || transform == Timestamps.MONTH + || transform == Timestamps.MICROS_TO_MONTH + || transform == Timestamps.NANOS_TO_MONTH || transform instanceof Months) { return visitor.month(field.fieldId(), sourceName, field.sourceId()); - } else if (transform == Dates.DAY || transform == Timestamps.DAY || transform instanceof Days) { + } else if (transform == Dates.DAY + || transform == Timestamps.MICROS_TO_DAY + || transform == Timestamps.NANOS_TO_DAY + || transform instanceof Days) { return visitor.day(field.fieldId(), sourceName, field.sourceId()); - } else if (transform == Timestamps.HOUR || transform instanceof Hours) { + } else if (transform == Timestamps.MICROS_TO_HOUR + || transform == Timestamps.NANOS_TO_HOUR + || transform instanceof Hours) { return visitor.hour(field.fieldId(), sourceName, field.sourceId()); } else if (transform instanceof VoidTransform) { return visitor.alwaysNull(field.fieldId(), sourceName, field.sourceId()); diff --git a/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java index 680e095270fb..62cc9d3cdb33 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java @@ -85,21 +85,26 @@ static <R> List<R> visit(SortOrder sortOrder, SortOrderVisitor<R> visitor) { visitor.truncate( sourceName, field.sourceId(), width, field.direction(), field.nullOrder())); } else if (transform == Dates.YEAR - || transform == Timestamps.YEAR + || transform == Timestamps.MICROS_TO_YEAR + || transform == Timestamps.NANOS_TO_YEAR || transform instanceof Years) { results.add( visitor.year(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform == Dates.MONTH - || transform == Timestamps.MONTH + || transform == Timestamps.MICROS_TO_MONTH + || transform == Timestamps.NANOS_TO_MONTH || transform instanceof Months) { results.add( visitor.month(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform == Dates.DAY - || transform == Timestamps.DAY + || transform == Timestamps.MICROS_TO_DAY + || transform == Timestamps.NANOS_TO_DAY || transform instanceof Days) { results.add( visitor.day(sourceName, field.sourceId(), field.direction(), field.nullOrder())); - } else if (transform == Timestamps.HOUR || transform instanceof Hours) { + } else if (transform == Timestamps.MICROS_TO_HOUR + || transform == Timestamps.NANOS_TO_HOUR + || transform instanceof Hours) { results.add( visitor.hour(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform instanceof UnknownTransform) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java b/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java index 01ea8130aa60..c348fda52b02 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.transforms; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.expressions.BoundPredicate; import org.apache.iceberg.expressions.BoundTransform; import org.apache.iceberg.expressions.UnboundPredicate; @@ -25,6 +26,24 @@ import org.apache.iceberg.util.SerializableFunction; abstract class TimeTransform<S> implements Transform<S, Integer> { + protected static <R> R fromSourceType(Type type, R dateResult, R microsResult, R nanosResult) { + switch (type.typeId()) { + case DATE: + if (dateResult != null) { + return dateResult; + } + break; + case TIMESTAMP: + return microsResult; + case TIMESTAMP_NANO: + return nanosResult; + } + + throw new IllegalArgumentException("Unsupported type: " + type); + } + + protected abstract ChronoUnit granularity(); + protected abstract Transform<S, Integer> toEnum(Type type); @Override @@ -37,9 +56,29 @@ public boolean preservesOrder() { return true; } + @Override + public boolean satisfiesOrderOf(Transform<?, ?> other) { + if (this == other) { + return true; + } + + if (other instanceof Dates) { + return TransformUtil.satisfiesOrderOf(granularity(), ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity(), ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf( + granularity(), ((TimeTransform<?>) other).granularity()); + } + + return false; + } + @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.DATE || type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.DATE + || type.typeId() == Type.TypeID.TIMESTAMP + || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java index b5b50e9d42b2..8b8c2ca0a96b 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java @@ -32,53 +32,29 @@ import org.apache.iceberg.util.SerializableFunction; enum Timestamps implements Transform<Long, Integer> { - YEAR(ChronoUnit.YEARS, "year"), - MONTH(ChronoUnit.MONTHS, "month"), - DAY(ChronoUnit.DAYS, "day"), - HOUR(ChronoUnit.HOURS, "hour"); + MICROS_TO_YEAR(ChronoUnit.YEARS, "year", MicrosToYears.INSTANCE), + MICROS_TO_MONTH(ChronoUnit.MONTHS, "month", MicrosToMonths.INSTANCE), + MICROS_TO_DAY(ChronoUnit.DAYS, "day", MicrosToDays.INSTANCE), + MICROS_TO_HOUR(ChronoUnit.HOURS, "hour", MicrosToHours.INSTANCE), - @Immutable - static class Apply implements SerializableFunction<Long, Integer> { - private final ChronoUnit granularity; - - Apply(ChronoUnit granularity) { - this.granularity = granularity; - } - - @Override - public Integer apply(Long timestampMicros) { - if (timestampMicros == null) { - return null; - } - - switch (granularity) { - case YEARS: - return DateTimeUtil.microsToYears(timestampMicros); - case MONTHS: - return DateTimeUtil.microsToMonths(timestampMicros); - case DAYS: - return DateTimeUtil.microsToDays(timestampMicros); - case HOURS: - return DateTimeUtil.microsToHours(timestampMicros); - default: - throw new UnsupportedOperationException("Unsupported time unit: " + granularity); - } - } - } + NANOS_TO_YEAR(ChronoUnit.YEARS, "year", NanosToYears.INSTANCE), + NANOS_TO_MONTH(ChronoUnit.MONTHS, "month", NanosToMonths.INSTANCE), + NANOS_TO_DAY(ChronoUnit.DAYS, "day", NanosToDays.INSTANCE), + NANOS_TO_HOUR(ChronoUnit.HOURS, "hour", NanosToHours.INSTANCE); private final ChronoUnit granularity; private final String name; - private final Apply apply; + private final SerializableFunction<Long, Integer> apply; - Timestamps(ChronoUnit granularity, String name) { - this.granularity = granularity; + Timestamps(ChronoUnit granularity, String name, SerializableFunction<Long, Integer> apply) { this.name = name; - this.apply = new Apply(granularity); + this.granularity = granularity; + this.apply = apply; } @Override - public Integer apply(Long timestampMicros) { - return apply.apply(timestampMicros); + public Integer apply(Long timestamp) { + return apply.apply(timestamp); } @Override @@ -89,7 +65,7 @@ public SerializableFunction<Long, Integer> bind(Type type) { @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.TIMESTAMP || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override @@ -100,6 +76,10 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } + ChronoUnit granularity() { + return granularity; + } + @Override public boolean preservesOrder() { return true; @@ -111,12 +91,12 @@ public boolean satisfiesOrderOf(Transform<?, ?> other) { return true; } - if (other instanceof Timestamps) { - // test the granularity, in hours. hour(ts) => 1 hour, day(ts) => 24 hours, and hour satisfies - // the order of day - Timestamps otherTransform = (Timestamps) other; - return granularity.getDuration().toHours() - <= otherTransform.granularity.getDuration().toHours(); + if (other instanceof Dates) { + return TransformUtil.satisfiesOrderOf(granularity, ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity, ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf(granularity, ((TimeTransform<?>) other).granularity()); } return false; @@ -197,4 +177,116 @@ public String toString() { public String dedupName() { return "time"; } + + @Immutable + static class MicrosToYears implements SerializableFunction<Long, Integer> { + static final MicrosToYears INSTANCE = new MicrosToYears(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToYears(micros); + } + } + + @Immutable + static class MicrosToMonths implements SerializableFunction<Long, Integer> { + static final MicrosToMonths INSTANCE = new MicrosToMonths(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToMonths(micros); + } + } + + @Immutable + static class MicrosToDays implements SerializableFunction<Long, Integer> { + static final MicrosToDays INSTANCE = new MicrosToDays(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToDays(micros); + } + } + + @Immutable + static class MicrosToHours implements SerializableFunction<Long, Integer> { + static final MicrosToHours INSTANCE = new MicrosToHours(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToHours(micros); + } + } + + @Immutable + static class NanosToYears implements SerializableFunction<Long, Integer> { + static final NanosToYears INSTANCE = new NanosToYears(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToYears(nanos); + } + } + + @Immutable + static class NanosToMonths implements SerializableFunction<Long, Integer> { + static final NanosToMonths INSTANCE = new NanosToMonths(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToMonths(nanos); + } + } + + @Immutable + static class NanosToDays implements SerializableFunction<Long, Integer> { + static final NanosToDays INSTANCE = new NanosToDays(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToDays(nanos); + } + } + + @Immutable + static class NanosToHours implements SerializableFunction<Long, Integer> { + static final NanosToHours INSTANCE = new NanosToHours(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToHours(nanos); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transform.java b/api/src/main/java/org/apache/iceberg/transforms/Transform.java index 5a56b672b1b1..78312b58b12f 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transform.java @@ -181,6 +181,12 @@ default String toHumanString(Type type, T value) { } else { return TransformUtil.humanTimestampWithoutZone((Long) value); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return TransformUtil.humanTimestampNanoWithZone((Long) value); + } else { + return TransformUtil.humanTimestampNanoWithoutZone((Long) value); + } case FIXED: case BINARY: if (value instanceof ByteBuffer) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java index 53bc23a49888..dd7f97e950e8 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java @@ -26,6 +26,7 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.Base64; +import org.apache.iceberg.util.DateTimeUtil; class TransformUtil { @@ -55,11 +56,19 @@ static String humanTime(Long microsFromMidnight) { } static String humanTimestampWithZone(Long timestampMicros) { - return ChronoUnit.MICROS.addTo(EPOCH, timestampMicros).toString(); + return DateTimeUtil.microsToIsoTimestamptz(timestampMicros); } static String humanTimestampWithoutZone(Long timestampMicros) { - return ChronoUnit.MICROS.addTo(EPOCH, timestampMicros).toLocalDateTime().toString(); + return DateTimeUtil.microsToIsoTimestamp(timestampMicros); + } + + static String humanTimestampNanoWithZone(Long timestampNanos) { + return DateTimeUtil.nanosToIsoTimestamptz(timestampNanos); + } + + static String humanTimestampNanoWithoutZone(Long timestampNanos) { + return DateTimeUtil.nanosToIsoTimestamp(timestampNanos); } static String humanHour(int hourOrdinal) { @@ -73,4 +82,10 @@ static String base64encode(ByteBuffer buffer) { // use direct encoding because all of the encoded bytes are in ASCII return StandardCharsets.ISO_8859_1.decode(Base64.getEncoder().encode(buffer)).toString(); } + + static boolean satisfiesOrderOf(ChronoUnit leftGranularity, ChronoUnit rightGranularity) { + // test the granularity, in hours. hour(ts) => 1 hour, day(ts) => 24 hours, and hour satisfies + // the order of day + return leftGranularity.getDuration().toHours() <= rightGranularity.getDuration().toHours(); + } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java index a1ce33ddd6da..11282efdefb1 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java @@ -23,7 +23,6 @@ import java.util.regex.Pattern; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Type; /** @@ -68,6 +67,8 @@ private Transforms() {} return new UnknownTransform<>(transform); } + /** @deprecated use {@link #identity()} instead; will be removed in 2.0.0 */ + @Deprecated public static Transform<?, ?> fromString(Type type, String transform) { Matcher widthMatcher = HAS_WIDTH.matcher(transform); if (widthMatcher.matches()) { @@ -80,22 +81,20 @@ private Transforms() {} } } - if (transform.equalsIgnoreCase("identity")) { - return Identity.get(type); - } - - try { - if (type.typeId() == Type.TypeID.TIMESTAMP) { - return Timestamps.valueOf(transform.toUpperCase(Locale.ENGLISH)); - } else if (type.typeId() == Type.TypeID.DATE) { - return Dates.valueOf(transform.toUpperCase(Locale.ENGLISH)); - } - } catch (IllegalArgumentException ignored) { - // fall through to return unknown transform - } - - if (transform.equalsIgnoreCase("void")) { - return VoidTransform.get(); + String lowerTransform = transform.toLowerCase(Locale.ENGLISH); + switch (lowerTransform) { + case "identity": + return Identity.get(type); + case "year": + return Years.get().toEnum(type); + case "month": + return Months.get().toEnum(type); + case "day": + return Days.get().toEnum(type); + case "hour": + return Hours.get().toEnum(type); + case "void": + return VoidTransform.get(); } return new UnknownTransform<>(transform); @@ -125,14 +124,7 @@ public static <T> Transform<T, T> identity(Type type) { @Deprecated @SuppressWarnings("unchecked") public static <T> Transform<T, Integer> year(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform<T, Integer>) Dates.YEAR; - case TIMESTAMP: - return (Transform<T, Integer>) Timestamps.YEAR; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by year"); - } + return (Transform<T, Integer>) Years.get().toEnum(type); } /** @@ -146,14 +138,7 @@ public static <T> Transform<T, Integer> year(Type type) { @Deprecated @SuppressWarnings("unchecked") public static <T> Transform<T, Integer> month(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform<T, Integer>) Dates.MONTH; - case TIMESTAMP: - return (Transform<T, Integer>) Timestamps.MONTH; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by month"); - } + return (Transform<T, Integer>) Months.get().toEnum(type); } /** @@ -167,14 +152,7 @@ public static <T> Transform<T, Integer> month(Type type) { @Deprecated @SuppressWarnings("unchecked") public static <T> Transform<T, Integer> day(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform<T, Integer>) Dates.DAY; - case TIMESTAMP: - return (Transform<T, Integer>) Timestamps.DAY; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by day"); - } + return (Transform<T, Integer>) Days.get().toEnum(type); } /** @@ -188,9 +166,7 @@ public static <T> Transform<T, Integer> day(Type type) { @Deprecated @SuppressWarnings("unchecked") public static <T> Transform<T, Integer> hour(Type type) { - Preconditions.checkArgument( - type.typeId() == Type.TypeID.TIMESTAMP, "Cannot partition type %s by hour", type); - return (Transform<T, Integer>) Timestamps.HOUR; + return (Transform<T, Integer>) Hours.get().toEnum(type); } /** diff --git a/api/src/main/java/org/apache/iceberg/transforms/Years.java b/api/src/main/java/org/apache/iceberg/transforms/Years.java index 6c1eee578506..2920a37dc692 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Years.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Years.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static <T> Years<T> get() { } @Override - @SuppressWarnings("unchecked") - protected Transform<T, Integer> toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform<T, Integer>) Dates.YEAR; - case TIMESTAMP: - return (Transform<T, Integer>) Timestamps.YEAR; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.YEARS; } @Override - public Type getResultType(Type sourceType) { - return Types.IntegerType.get(); + protected Transform<T, Integer> toEnum(Type type) { + return (Transform<T, Integer>) + fromSourceType(type, Dates.YEAR, Timestamps.MICROS_TO_YEAR, Timestamps.NANOS_TO_YEAR); } @Override - public boolean satisfiesOrderOf(Transform<?, ?> other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.YEAR.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.YEAR.satisfiesOrderOf(other); - } else if (other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.IntegerType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java index d09d9f5395ce..bfbffc64b673 100644 --- a/api/src/main/java/org/apache/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -41,6 +41,8 @@ private Comparators() {} .put(Types.TimeType.get(), Comparator.naturalOrder()) .put(Types.TimestampType.withZone(), Comparator.naturalOrder()) .put(Types.TimestampType.withoutZone(), Comparator.naturalOrder()) + .put(Types.TimestampNanoType.withZone(), Comparator.naturalOrder()) + .put(Types.TimestampNanoType.withoutZone(), Comparator.naturalOrder()) .put(Types.StringType.get(), Comparators.charSequences()) .put(Types.UUIDType.get(), Comparator.naturalOrder()) .put(Types.BinaryType.get(), Comparators.unsignedBytes()) @@ -173,6 +175,10 @@ public static Comparator<CharSequence> charSequences() { return CharSeqComparator.INSTANCE; } + public static Comparator<CharSequence> filePath() { + return FilePathComparator.INSTANCE; + } + private static class NullsFirst<T> implements Comparator<T> { private static final NullsFirst<?> INSTANCE = new NullsFirst<>(); @@ -351,4 +357,41 @@ public int compare(CharSequence s1, CharSequence s2) { return Integer.compare(s1.length(), s2.length()); } } + + private static class FilePathComparator implements Comparator<CharSequence> { + 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/api/src/main/java/org/apache/iceberg/types/Conversions.java b/api/src/main/java/org/apache/iceberg/types/Conversions.java index 1d2539514954..e18c7b4362e6 100644 --- a/api/src/main/java/org/apache/iceberg/types/Conversions.java +++ b/api/src/main/java/org/apache/iceberg/types/Conversions.java @@ -97,6 +97,7 @@ public static ByteBuffer toByteBuffer(Type.TypeID typeId, Object value) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, (long) value); case FLOAT: return ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN).putFloat(0, (float) value); @@ -146,6 +147,7 @@ private static Object internalFromByteBuffer(Type type, ByteBuffer buffer) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: if (tmp.remaining() < 8) { // type was later promoted to long return (long) tmp.getInt(); diff --git a/api/src/main/java/org/apache/iceberg/types/Type.java b/api/src/main/java/org/apache/iceberg/types/Type.java index 5062b54d10e1..571bf9a14e43 100644 --- a/api/src/main/java/org/apache/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -37,6 +37,7 @@ enum TypeID { DATE(Integer.class), TIME(Long.class), TIMESTAMP(Long.class), + TIMESTAMP_NANO(Long.class), STRING(CharSequence.class), UUID(java.util.UUID.class), FIXED(ByteBuffer.class), diff --git a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java index 07d06dcc5a89..9d4b217f1595 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -181,11 +181,37 @@ public static Map<Integer, String> indexQuotedNameById( return indexer.byId(); } + /** + * Creates a mapping from lower-case field names to their corresponding field IDs. + * + * <p>This method iterates over the fields of the provided struct and maps each field's name + * (converted to lower-case) to its ID. If two fields have the same lower-case name, an + * `IllegalArgumentException` is thrown. + * + * @param struct the struct type whose fields are to be indexed + * @return a map where the keys are lower-case field names and the values are field IDs + * @throws IllegalArgumentException if two fields have the same lower-case name + */ public static Map<String, Integer> indexByLowerCaseName(Types.StructType struct) { Map<String, Integer> indexByLowerCaseName = Maps.newHashMap(); + + IndexByName indexer = new IndexByName(); + visit(struct, indexer); + Map<String, Integer> byName = indexer.byName(); + Map<Integer, String> byId = indexer.byId(); + indexByName(struct) .forEach( - (name, integer) -> indexByLowerCaseName.put(name.toLowerCase(Locale.ROOT), integer)); + (name, fieldId) -> { + String key = name.toLowerCase(Locale.ROOT); + Integer existingId = indexByLowerCaseName.put(key, fieldId); + Preconditions.checkArgument( + existingId == null || existingId.equals(fieldId), + "Cannot build lower case index: %s and %s collide", + byId.get(existingId), + byId.get(fieldId)); + indexByLowerCaseName.put(key, fieldId); + }); return indexByLowerCaseName; } @@ -496,6 +522,7 @@ private static int estimateSize(Type type) { case DOUBLE: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: // longs and doubles occupy 8 bytes // times and timestamps are internally represented as longs return 8; diff --git a/api/src/main/java/org/apache/iceberg/types/Types.java b/api/src/main/java/org/apache/iceberg/types/Types.java index ce6caa4721df..2352b9b52f13 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -49,6 +49,8 @@ private Types() {} .put(TimeType.get().toString(), TimeType.get()) .put(TimestampType.withZone().toString(), TimestampType.withZone()) .put(TimestampType.withoutZone().toString(), TimestampType.withoutZone()) + .put(TimestampNanoType.withZone().toString(), TimestampNanoType.withZone()) + .put(TimestampNanoType.withoutZone().toString(), TimestampNanoType.withoutZone()) .put(StringType.get().toString(), StringType.get()) .put(UUIDType.get().toString(), UUIDType.get()) .put(BinaryType.get().toString(), BinaryType.get()) @@ -259,6 +261,59 @@ public int hashCode() { } } + public static class TimestampNanoType extends PrimitiveType { + private static final TimestampNanoType INSTANCE_WITH_ZONE = new TimestampNanoType(true); + private static final TimestampNanoType INSTANCE_WITHOUT_ZONE = new TimestampNanoType(false); + + public static TimestampNanoType withZone() { + return INSTANCE_WITH_ZONE; + } + + public static TimestampNanoType withoutZone() { + return INSTANCE_WITHOUT_ZONE; + } + + private final boolean adjustToUTC; + + private TimestampNanoType(boolean adjustToUTC) { + this.adjustToUTC = adjustToUTC; + } + + public boolean shouldAdjustToUTC() { + return adjustToUTC; + } + + @Override + public TypeID typeId() { + return TypeID.TIMESTAMP_NANO; + } + + @Override + public String toString() { + if (shouldAdjustToUTC()) { + return "timestamptz_ns"; + } else { + return "timestamp_ns"; + } + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof TimestampNanoType)) { + return false; + } + + return adjustToUTC == ((TimestampNanoType) other).adjustToUTC; + } + + @Override + public int hashCode() { + return Objects.hash(TimestampNanoType.class, adjustToUTC); + } + } + public static class StringType extends PrimitiveType { private static final StringType INSTANCE = new StringType(); diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java index 1bb5a1dc4e87..cfdac0104c47 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java @@ -30,7 +30,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Streams; public class CharSequenceSet implements Set<CharSequence>, Serializable { - private static final ThreadLocal<CharSequenceWrapper> wrappers = + private static final ThreadLocal<CharSequenceWrapper> WRAPPERS = ThreadLocal.withInitial(() -> CharSequenceWrapper.wrap(null)); public static CharSequenceSet of(Iterable<CharSequence> charSequences) { @@ -61,7 +61,7 @@ public boolean isEmpty() { @Override public boolean contains(Object obj) { if (obj instanceof CharSequence) { - CharSequenceWrapper wrapper = wrappers.get(); + CharSequenceWrapper wrapper = WRAPPERS.get(); boolean result = wrapperSet.contains(wrapper.set((CharSequence) obj)); wrapper.set(null); // don't hold a reference to the value return result; @@ -109,7 +109,7 @@ public boolean add(CharSequence charSequence) { @Override public boolean remove(Object obj) { if (obj instanceof CharSequence) { - CharSequenceWrapper wrapper = wrappers.get(); + CharSequenceWrapper wrapper = WRAPPERS.get(); boolean result = wrapperSet.remove(wrapper.set((CharSequence) obj)); wrapper.set(null); // don't hold a reference to the value return result; diff --git a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index a2f5301f44a9..3c312486be00 100644 --- a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -35,6 +35,15 @@ private DateTimeUtil() {} public static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); public static final long MICROS_PER_MILLIS = 1000L; public static final long MICROS_PER_SECOND = 1_000_000L; + private static final long NANOS_PER_SECOND = 1_000_000_000L; + private static final long NANOS_PER_MICRO = 1_000L; + + private static final DateTimeFormatter FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HH:MM:ss", "+00:00") + .toFormatter(); public static LocalDate dateFromDays(int daysFromEpoch) { return ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); @@ -60,6 +69,10 @@ public static LocalDateTime timestampFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch).toLocalDateTime(); } + public static LocalDateTime timestampFromNanos(long nanosFromEpoch) { + return ChronoUnit.NANOS.addTo(EPOCH, nanosFromEpoch).toLocalDateTime(); + } + public static long microsFromInstant(Instant instant) { return ChronoUnit.MICROS.between(EPOCH, instant.atOffset(ZoneOffset.UTC)); } @@ -68,6 +81,10 @@ public static long microsFromTimestamp(LocalDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); } + public static long nanosFromTimestamp(LocalDateTime dateTime) { + return ChronoUnit.NANOS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); + } + public static long microsToMillis(long micros) { // When the timestamp is negative, i.e before 1970, we need to adjust the milliseconds portion. // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. @@ -75,6 +92,14 @@ public static long microsToMillis(long micros) { return Math.floorDiv(micros, MICROS_PER_MILLIS); } + public static long nanosToMicros(long nanos) { + return Math.floorDiv(nanos, NANOS_PER_MICRO); + } + + public static long microsToNanos(long micros) { + return Math.multiplyExact(micros, NANOS_PER_MICRO); + } + public static OffsetDateTime timestamptzFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch); } @@ -83,6 +108,10 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime); } + public static long nanosFromTimestamptz(OffsetDateTime dateTime) { + return ChronoUnit.NANOS.between(EPOCH, dateTime); + } + public static String formatTimestampMillis(long millis) { return Instant.ofEpochMilli(millis).toString().replace("Z", "+00:00"); } @@ -97,13 +126,12 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - DateTimeFormatter zeroOffsetFormatter = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .appendOffset("+HH:MM:ss", "+00:00") - .toFormatter(); - return localDateTime.atOffset(ZoneOffset.UTC).format(zeroOffsetFormatter); + return localDateTime.atOffset(ZoneOffset.UTC).format(FORMATTER); + } + + public static String nanosToIsoTimestamptz(long nanos) { + LocalDateTime localDateTime = timestampFromNanos(nanos); + return localDateTime.atOffset(ZoneOffset.UTC).format(FORMATTER); } public static String microsToIsoTimestamp(long micros) { @@ -111,6 +139,11 @@ public static String microsToIsoTimestamp(long micros) { return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); } + public static String nanosToIsoTimestamp(long nanos) { + LocalDateTime localDateTime = timestampFromNanos(nanos); + return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + } + public static int isoDateToDays(String dateString) { return daysFromDate(LocalDate.parse(dateString, DateTimeFormatter.ISO_LOCAL_DATE)); } @@ -124,6 +157,11 @@ public static long isoTimestamptzToMicros(String timestampString) { OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } + public static long isoTimestamptzToNanos(CharSequence timestampString) { + return nanosFromTimestamptz( + OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); + } + public static boolean isUTCTimestamptz(String timestampString) { OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); @@ -135,6 +173,11 @@ public static long isoTimestampToMicros(String timestampString) { LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } + public static long isoTimestampToNanos(CharSequence timestampString) { + return nanosFromTimestamp( + LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } + public static int daysToYears(int days) { return convertDays(days, ChronoUnit.YEARS); } @@ -185,6 +228,36 @@ private static int convertMicros(long micros, ChronoUnit granularity) { } } + public static int nanosToYears(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.YEARS)); + } + + public static int nanosToMonths(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.MONTHS)); + } + + public static int nanosToDays(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.DAYS)); + } + + public static int nanosToHours(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.HOURS)); + } + + private static long convertNanos(long nanos, ChronoUnit granularity) { + if (nanos >= 0) { + long epochSecond = Math.floorDiv(nanos, NANOS_PER_SECOND); + long nanoAdjustment = Math.floorMod(nanos, NANOS_PER_SECOND); + return granularity.between(EPOCH, toOffsetDateTime(epochSecond, nanoAdjustment)); + } else { + // add 1 nano to the value to account for the case where there is exactly 1 unit between + // the timestamp and epoch because the result will always be decremented. + long epochSecond = Math.floorDiv(nanos, NANOS_PER_SECOND); + long nanoAdjustment = Math.floorMod(nanos + 1, NANOS_PER_SECOND); + return granularity.between(EPOCH, toOffsetDateTime(epochSecond, nanoAdjustment)) - 1; + } + } + private static OffsetDateTime toOffsetDateTime(long epochSecond, long nanoAdjustment) { return Instant.ofEpochSecond(epochSecond, nanoAdjustment).atOffset(ZoneOffset.UTC); } diff --git a/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java b/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java index 5e4ca1fb11be..11f2cb353880 100644 --- a/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java +++ b/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java @@ -34,7 +34,8 @@ public class PartitionSpecTestBase { Types.NestedField.required(7, "s", Types.StringType.get()), Types.NestedField.required(8, "u", Types.UUIDType.get()), Types.NestedField.required(9, "f", Types.FixedType.ofLength(3)), - Types.NestedField.required(10, "b", Types.BinaryType.get())); + Types.NestedField.required(10, "b", Types.BinaryType.get()), + Types.NestedField.required(11, "tsn", Types.TimestampNanoType.withoutZone())); // a spec with all of the allowed transform/type pairs public static final PartitionSpec[] SPECS = @@ -49,6 +50,7 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).identity("u").build(), PartitionSpec.builderFor(SCHEMA).identity("f").build(), PartitionSpec.builderFor(SCHEMA).identity("b").build(), + PartitionSpec.builderFor(SCHEMA).identity("tsn").build(), PartitionSpec.builderFor(SCHEMA).bucket("i", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("l", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("d", 128).build(), @@ -59,6 +61,7 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).bucket("u", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("f", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("b", 128).build(), + PartitionSpec.builderFor(SCHEMA).bucket("tsn", 128).build(), PartitionSpec.builderFor(SCHEMA).year("d").build(), PartitionSpec.builderFor(SCHEMA).month("d").build(), PartitionSpec.builderFor(SCHEMA).day("d").build(), @@ -66,6 +69,10 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).month("ts").build(), PartitionSpec.builderFor(SCHEMA).day("ts").build(), PartitionSpec.builderFor(SCHEMA).hour("ts").build(), + PartitionSpec.builderFor(SCHEMA).year("tsn").build(), + PartitionSpec.builderFor(SCHEMA).month("tsn").build(), + PartitionSpec.builderFor(SCHEMA).day("tsn").build(), + PartitionSpec.builderFor(SCHEMA).hour("tsn").build(), PartitionSpec.builderFor(SCHEMA).truncate("i", 10).build(), PartitionSpec.builderFor(SCHEMA).truncate("l", 10).build(), PartitionSpec.builderFor(SCHEMA).truncate("dec", 10).build(), diff --git a/api/src/test/java/org/apache/iceberg/TestAccessors.java b/api/src/test/java/org/apache/iceberg/TestAccessors.java index 332556e474c7..7b4feb845f12 100644 --- a/api/src/test/java/org/apache/iceberg/TestAccessors.java +++ b/api/src/test/java/org/apache/iceberg/TestAccessors.java @@ -180,6 +180,8 @@ public void testTime() { public void testTimestamp() { assertAccessorReturns(Types.TimestampType.withoutZone(), 123L); assertAccessorReturns(Types.TimestampType.withZone(), 123L); + assertAccessorReturns(Types.TimestampNanoType.withoutZone(), 123L); + assertAccessorReturns(Types.TimestampNanoType.withZone(), 123L); } @Test diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java index 5455415da015..31a6c486bf6e 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java @@ -54,6 +54,44 @@ public void testPartitionPath() { .isEqualTo("ts_hour=2017-12-01-10/id_bucket=" + idBucket); } + @Test + public void testPartitionPathWithNanoseconds() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("id", 10).build(); + + Transform<Long, Integer> hour = Transforms.hour(); + Transform<Integer, Integer> bucket = Transforms.bucket(10); + + Literal<Long> ts = + Literal.of("2017-12-01T10:12:55.038194789").to(Types.TimestampNanoType.withoutZone()); + Object tsHour = hour.bind(Types.TimestampNanoType.withoutZone()).apply(ts.value()); + Object idBucket = bucket.bind(Types.IntegerType.get()).apply(1); + + Row partition = Row.of(tsHour, idBucket); + + assertThat(spec.partitionToPath(partition)) + .as("Should produce expected partition key") + .isEqualTo("ts_hour=2017-12-01-10/id_bucket=" + idBucket); + } + + @Test + public void testPartitionPathWithNanosecondsTz() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("id", 10).build(); + + Transform<Long, Integer> hour = Transforms.hour(); + Transform<Integer, Integer> bucket = Transforms.bucket(10); + + Literal<Long> ts = + Literal.of("2017-12-01T10:12:55.038194789-08:00").to(Types.TimestampNanoType.withZone()); + Object tsTzHour = hour.bind(Types.TimestampNanoType.withZone()).apply(ts.value()); + Object idBucket = bucket.bind(Types.IntegerType.get()).apply(1); + + Row partition = Row.of(tsTzHour, idBucket); + + assertThat(spec.partitionToPath(partition)) + .as("Should produce expected partition key") + .isEqualTo("ts_hour=2017-12-01-18/id_bucket=" + idBucket); + } + @Test public void testEscapedStrings() { PartitionSpec spec = diff --git a/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java b/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java new file mode 100644 index 000000000000..bdb73374bf21 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSchemaCaseSensitivity { + + @Test + public void testCaseInsensitiveFieldCollision() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get())); + assertThatIllegalArgumentException() + .isThrownBy(() -> schema.caseInsensitiveFindField("DATA")) + .withMessage("Cannot build lower case index: data and DATA collide"); + } + + @Test + public void testCaseSensitiveFindField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get())); + + Types.NestedField actual1 = schema.findField("data"); + assertThat(actual1).isEqualTo(Types.NestedField.required(2, "data", Types.StringType.get())); + Types.NestedField actual2 = schema.findField("DATA"); + assertThat(actual2).isEqualTo(Types.NestedField.required(3, "DATA", Types.StringType.get())); + } + + @Test + public void testCaseInsensitiveField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); + + Types.NestedField actual1 = schema.caseInsensitiveFindField("DATA"); + assertThat(actual1).isEqualTo(Types.NestedField.required(2, "data", Types.StringType.get())); + } +} diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java index 95a9ac2cc83a..23c15b5461e7 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java @@ -29,15 +29,15 @@ import org.junit.jupiter.api.Test; public class TestAggregateBinding { - private static final List<UnboundAggregate<Integer>> list = + private static final List<UnboundAggregate<Integer>> LIST = ImmutableList.of(Expressions.count("x"), Expressions.max("x"), Expressions.min("x")); - private static final StructType struct = + private static final StructType STRUCT = StructType.of(Types.NestedField.required(10, "x", Types.IntegerType.get())); @Test public void testAggregateBinding() { - for (UnboundAggregate<Integer> unbound : list) { - Expression expr = unbound.bind(struct, true); + for (UnboundAggregate<Integer> unbound : LIST) { + Expression expr = unbound.bind(STRUCT, true); BoundAggregate<Integer, ?> bound = assertAndUnwrapAggregate(expr); assertThat(bound.ref().fieldId()).as("Should reference correct field ID").isEqualTo(10); assertThat(bound.op()) @@ -60,7 +60,7 @@ public void testCountStarBinding() { @Test public void testBoundAggregateFails() { Expression unbound = Expressions.count("x"); - assertThatThrownBy(() -> Binder.bind(struct, Binder.bind(struct, unbound))) + assertThatThrownBy(() -> Binder.bind(STRUCT, Binder.bind(STRUCT, unbound))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Found already bound aggregate"); } @@ -68,7 +68,7 @@ public void testBoundAggregateFails() { @Test public void testCaseInsensitiveReference() { Expression expr = Expressions.max("X"); - Expression boundExpr = Binder.bind(struct, expr, false); + Expression boundExpr = Binder.bind(STRUCT, expr, false); BoundAggregate<Integer, Integer> bound = assertAndUnwrapAggregate(boundExpr); assertThat(bound.ref().fieldId()).as("Should reference correct field ID").isEqualTo(10); assertThat(bound.op()) @@ -79,7 +79,7 @@ public void testCaseInsensitiveReference() { @Test public void testCaseSensitiveReference() { Expression expr = Expressions.max("X"); - assertThatThrownBy(() -> Binder.bind(struct, expr, true)) + assertThatThrownBy(() -> Binder.bind(STRUCT, expr, true)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'X' in struct"); } @@ -87,7 +87,7 @@ public void testCaseSensitiveReference() { @Test public void testMissingField() { UnboundAggregate<?> unbound = Expressions.count("missing"); - assertThatThrownBy(() -> unbound.bind(struct, false)) + assertThatThrownBy(() -> unbound.bind(STRUCT, false)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing' in struct:"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java index b418dede8610..aa15d36de3f7 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java @@ -91,7 +91,7 @@ public class TestAggregateEvaluator { // upper bounds ImmutableMap.of(1, toByteBuffer(IntegerType.get(), 3333))); - private static final DataFile[] dataFiles = { + private static final DataFile[] DATA_FILES = { FILE, MISSING_SOME_NULLS_STATS_1, MISSING_SOME_NULLS_STATS_2 }; @@ -121,7 +121,7 @@ public void testIntAggregate() { Expressions.min("id")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } @@ -141,7 +141,7 @@ public void testAllNulls() { Expressions.min("all_nulls")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } @@ -160,7 +160,7 @@ public void testSomeNulls() { Expressions.max("some_nulls"), Expressions.min("some_nulls")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } @@ -179,7 +179,7 @@ public void testNoStats() { Expressions.max("no_stats"), Expressions.min("no_stats")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index 894989f38e7d..10d3b6d0adfa 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -42,11 +42,12 @@ public class TestExpressionUtil { Types.NestedField.required(2, "val", Types.IntegerType.get()), Types.NestedField.required(3, "val2", Types.IntegerType.get()), Types.NestedField.required(4, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.required(5, "date", Types.DateType.get()), - Types.NestedField.required(6, "time", Types.DateType.get()), - Types.NestedField.optional(7, "data", Types.StringType.get()), - Types.NestedField.optional(8, "measurement", Types.DoubleType.get()), - Types.NestedField.optional(9, "test", Types.IntegerType.get())); + Types.NestedField.required(5, "tsns", Types.TimestampNanoType.withoutZone()), + Types.NestedField.required(6, "date", Types.DateType.get()), + Types.NestedField.required(7, "time", Types.DateType.get()), + Types.NestedField.optional(8, "data", Types.StringType.get()), + Types.NestedField.optional(9, "measurement", Types.DoubleType.get()), + Types.NestedField.optional(10, "test", Types.IntegerType.get())); private static final Types.StructType STRUCT = SCHEMA.asStruct(); @@ -460,7 +461,9 @@ public void testSanitizeTimestamp() { "2022-04-29T23:49:51", "2022-04-29T23:49:51.123456", "2022-04-29T23:49:51-07:00", - "2022-04-29T23:49:51.123456+01:00")) { + "2022-04-29T23:49:51.123456+01:00", + "2022-04-29T23:49:51.123456789", + "2022-04-29T23:49:51.123456789+01:00")) { assertEquals( Expressions.equal("test", "(timestamp)"), ExpressionUtil.sanitize(Expressions.equal("test", timestamp))); @@ -496,6 +499,13 @@ public void testSanitizeTimestampAboutNow() { Expression.Operation.EQ, "test", Literal.of(nowLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-about-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(nowLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", nowLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -522,6 +532,13 @@ public void testSanitizeTimestampPast() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesAgoLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesAgoLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesAgoLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -548,6 +565,13 @@ public void testSanitizeTimestampLastWeek() { Expression.Operation.EQ, "test", Literal.of(lastWeekLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-7-days-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(lastWeekLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", lastWeekLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -574,6 +598,13 @@ public void testSanitizeTimestampFuture() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesFromNowLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-from-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesFromNowLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat( ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesFromNowLocal))) @@ -597,6 +628,13 @@ public void testSanitizeTimestamptzAboutNow() { Expression.Operation.EQ, "test", Literal.of(nowUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-about-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(nowUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", nowUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -618,6 +656,13 @@ public void testSanitizeTimestamptzPast() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesAgoUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesAgoUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesAgoUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -639,6 +684,13 @@ public void testSanitizeTimestamptzLastWeek() { Expression.Operation.EQ, "test", Literal.of(lastWeekUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-7-days-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(lastWeekUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", lastWeekUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -660,6 +712,13 @@ public void testSanitizeTimestamptzFuture() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesFromNowUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-from-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesFromNowUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesFromNowUtc))) .as("Sanitized string should be identical except for descriptive literal") diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java index d5aa251ffb50..24fc458b37b4 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java @@ -38,8 +38,10 @@ public void testLiterals() throws Exception { Literal.of(8.75D), Literal.of("2017-11-29").to(Types.DateType.get()), Literal.of("11:30:07").to(Types.TimeType.get()), - Literal.of("2017-11-29T11:30:07.123").to(Types.TimestampType.withoutZone()), - Literal.of("2017-11-29T11:30:07.123+01:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T11:30:07.123456").to(Types.TimestampType.withoutZone()), + Literal.of("2017-11-29T11:30:07.123456+01:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T11:30:07.123456789").to(Types.TimestampNanoType.withoutZone()), + Literal.of("2017-11-29T11:30:07.123456789+01:00").to(Types.TimestampNanoType.withZone()), Literal.of("abc"), Literal.of(UUID.randomUUID()), Literal.of(new byte[] {1, 2, 3}).to(Types.FixedType.ofLength(3)), diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java index f8d2cd49d969..e2611ddb281f 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java @@ -42,7 +42,9 @@ public void testIdentityConversions() { Pair.of(Literal.of("34.55"), Types.DecimalType.of(9, 2)), Pair.of(Literal.of("2017-08-18"), Types.DateType.get()), Pair.of(Literal.of("14:21:01.919"), Types.TimeType.get()), - Pair.of(Literal.of("2017-08-18T14:21:01.919"), Types.TimestampType.withoutZone()), + Pair.of(Literal.of("2017-08-18T14:21:01.919432"), Types.TimestampType.withoutZone()), + Pair.of( + Literal.of("2017-08-18T14:21:01.919432755"), Types.TimestampNanoType.withoutZone()), Pair.of(Literal.of("abc"), Types.StringType.get()), Pair.of(Literal.of(UUID.randomUUID()), Types.UUIDType.get()), Pair.of(Literal.of(new byte[] {0, 1, 2}), Types.FixedType.ofLength(3)), @@ -62,6 +64,22 @@ public void testIdentityConversions() { } } + @Test + public void testTimestampWithMicrosecondsToDate() { + final Literal<Long> micros = + Literal.of("2017-08-18T14:21:01.919432755").to(Types.TimestampType.withoutZone()); + final Literal<Integer> dateOfNanos = micros.to(Types.DateType.get()); + assertThat(dateOfNanos).isEqualTo(Literal.of("2017-08-18").to(Types.DateType.get())); + } + + @Test + public void testTimestampWithNanosecondsToDate() { + final Literal<Long> nanos = + Literal.of("2017-08-18T14:21:01.919432755").to(Types.TimestampNanoType.withoutZone()); + final Literal<Integer> dateOfNanos = nanos.to(Types.DateType.get()); + assertThat(dateOfNanos).isEqualTo(Literal.of("2017-08-18").to(Types.DateType.get())); + } + @Test public void testBinaryToFixed() { Literal<ByteBuffer> lit = Literal.of(ByteBuffer.wrap(new byte[] {0, 1, 2})); @@ -101,6 +119,8 @@ public void testInvalidBooleanConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), @@ -116,6 +136,8 @@ public void testInvalidIntegerConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -144,6 +166,8 @@ public void testInvalidFloatConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -161,6 +185,8 @@ public void testInvalidDoubleConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -179,6 +205,8 @@ public void testInvalidDateConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 4), Types.StringType.get(), Types.UUIDType.get(), @@ -198,6 +226,8 @@ public void testInvalidTimeConversions() { Types.DateType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 4), Types.StringType.get(), Types.UUIDType.get(), @@ -206,9 +236,26 @@ public void testInvalidTimeConversions() { } @Test - public void testInvalidTimestampConversions() { + public void testInvalidTimestampMicrosConversions() { testInvalidConversions( - Literal.of("2017-08-18T14:21:01.919").to(Types.TimestampType.withoutZone()), + Literal.of("2017-08-18T14:21:01.919123").to(Types.TimestampType.withoutZone()), + Types.BooleanType.get(), + Types.IntegerType.get(), + Types.LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.TimeType.get(), + Types.DecimalType.of(9, 4), + Types.StringType.get(), + Types.UUIDType.get(), + Types.FixedType.ofLength(1), + Types.BinaryType.get()); + } + + @Test + public void testInvalidTimestampNanosConversions() { + testInvalidConversions( + Literal.of("2017-08-18T14:21:01.919123456").to(Types.TimestampNanoType.withoutZone()), Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), @@ -233,8 +280,10 @@ public void testInvalidDecimalConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -267,8 +316,10 @@ public void testInvalidUUIDConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.FixedType.ofLength(1), @@ -286,8 +337,10 @@ public void testInvalidFixedConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), @@ -307,6 +360,8 @@ public void testInvalidBinaryConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java index 1dc2225b3805..45d6654f6aaf 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java @@ -132,6 +132,51 @@ public void testStringToTimestampLiteral() { .isEqualTo(avroValue); } + @Test + public void testStringToTimestampLiteralWithMicrosecondPrecisionFromNanoseconds() { + // use Avro's timestamp conversion to validate the result + Schema avroSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + TimeConversions.TimestampMicrosConversion avroConversion = + new TimeConversions.TimestampMicrosConversion(); + + Literal<CharSequence> timestampStr = Literal.of("2017-08-18T14:21:01.123456789"); + Literal<Long> timestamp = timestampStr.to(Types.TimestampType.withoutZone()); + long avroValue = + avroConversion.toLong( + LocalDateTime.of(2017, 8, 18, 14, 21, 1, 123456000).toInstant(ZoneOffset.UTC), + avroSchema, + avroSchema.getLogicalType()); + + assertThat((long) timestamp.value()) + .as("Timestamp without zone should match UTC") + .isEqualTo(avroValue); + } + + @Test + public void testStringToTimestampLiteralWithNanosecondPrecisionFromNanoseconds() { + Literal<CharSequence> timestampStr = Literal.of("2017-08-18T14:21:01.123456789"); + Literal<Long> timestamp = timestampStr.to(Types.TimestampNanoType.withoutZone()); + + // Not only using Avro's timestamp conversion as it has no timestampNanos(). + long expected = 1503066061123456789L; + assertThat((long) timestamp.value()) + .as("Timestamp without zone should match UTC") + .isEqualTo(expected); + + // use Avro's timestamp conversion to validate the result within one microsecond + Schema avroSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + TimeConversions.TimestampMicrosConversion avroConversion = + new TimeConversions.TimestampMicrosConversion(); + long avroValue = + avroConversion.toLong( + LocalDateTime.of(2017, 8, 18, 14, 21, 1, 123456000).toInstant(ZoneOffset.UTC), + avroSchema, + avroSchema.getLogicalType()); + assertThat(timestamp.value() - avroValue * 1000) + .as("Timestamp without zone should match UTC") + .isEqualTo(789L); + } + @Test public void testNegativeStringToTimestampLiteral() { // use Avro's timestamp conversion to validate the result @@ -181,8 +226,13 @@ public void testNegativeStringToTimestampLiteral() { @Test public void testTimestampWithZoneWithoutZoneInLiteral() { // Zone must be present in literals when converting to timestamp with zone - Literal<CharSequence> timestampStr = Literal.of("2017-08-18T14:21:01.919"); - assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withZone())) + assertThatThrownBy( + () -> Literal.of("2017-08-18T14:21:01.919").to(Types.TimestampType.withZone())) + .isInstanceOf(DateTimeException.class) + .hasMessageContaining("could not be parsed"); + assertThatThrownBy( + () -> + Literal.of("2017-08-18T14:21:01.919123456").to(Types.TimestampNanoType.withZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } @@ -190,8 +240,14 @@ public void testTimestampWithZoneWithoutZoneInLiteral() { @Test public void testTimestampWithoutZoneWithZoneInLiteral() { // Zone must not be present in literals when converting to timestamp without zone - Literal<CharSequence> timestampStr = Literal.of("2017-08-18T14:21:01.919+07:00"); - assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withoutZone())) + assertThatThrownBy( + () -> Literal.of("2017-08-18T14:21:01.919+07:00").to(Types.TimestampType.withoutZone())) + .isInstanceOf(DateTimeException.class) + .hasMessageContaining("could not be parsed"); + assertThatThrownBy( + () -> + Literal.of("2017-08-18T14:21:01.919123456+07:00") + .to(Types.TimestampNanoType.withoutZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java new file mode 100644 index 000000000000..379ad4db5e97 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.expressions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.format.DateTimeParseException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.jupiter.api.Test; + +public class TestTimestampLiteralConversions { + @Test + public void testTimestampToTimestampNanoConversion() { + Literal<Long> timestamp = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1510842668000000L); + + Literal<Long> timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(1510842668000000000L); + + timestamp = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(0L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(0L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1000L); + } + + @Test + public void testTimestampToDateConversion() { + Literal<Long> ts = + Literal.of("2017-11-16T14:31:08.000001").to(Types.TimestampType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + + ts = Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0).isEqualTo(DateTimeUtil.isoDateToDays("1970-01-01")); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0).isEqualTo(DateTimeUtil.isoDateToDays("1970-01-01")); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1).isEqualTo(DateTimeUtil.isoDateToDays("1969-12-31")); + } + + @Test + public void testTimestampMicrosToDateConversion() { + Literal<Long> ts = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + } + + @Test + public void testTimestampNanoToTimestampConversion() { + Literal<Long> timestamp = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1510842668000000001L); + + Literal<Long> timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(1510842668000000L); + + timestamp = + Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = + Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1L); + + timestamp = + Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1000L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1L); + } + + @Test + public void testTimestampNanosToDateConversion() { + Literal<Long> ts = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampNanoType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + } + + @Test + public void testTimestampNanosWithZoneConversion() { + Literal<CharSequence> isoTimestampNanosWithZoneOffset = + Literal.of("2017-11-16T14:31:08.000000001+00:00"); + + assertThatThrownBy(() -> isoTimestampNanosWithZoneOffset.to(Types.TimestampType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampNanosWithZoneOffset.to(Types.TimestampNanoType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampNanosWithZoneOffset.to(Types.TimestampType.withZone()).value()) + .isEqualTo(1510842668000000L); + + assertThat(isoTimestampNanosWithZoneOffset.to(Types.TimestampNanoType.withZone()).value()) + .isEqualTo(1510842668000000001L); + } + + @Test + public void testTimestampMicrosWithZoneConversion() { + Literal<CharSequence> isoTimestampMicrosWithZoneOffset = + Literal.of("2017-11-16T14:31:08.000001+00:00"); + + assertThatThrownBy(() -> isoTimestampMicrosWithZoneOffset.to(Types.TimestampType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampMicrosWithZoneOffset.to(Types.TimestampNanoType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampMicrosWithZoneOffset.to(Types.TimestampType.withZone()).value()) + .isEqualTo(1510842668000001L); + + assertThat(isoTimestampMicrosWithZoneOffset.to(Types.TimestampNanoType.withZone()).value()) + .isEqualTo(1510842668000001000L); + } + + @Test + public void testTimestampNanosWithoutZoneConversion() { + Literal<CharSequence> isoTimestampNanosWithoutZoneOffset = + Literal.of("2017-11-16T14:31:08.000000001"); + + assertThatThrownBy(() -> isoTimestampNanosWithoutZoneOffset.to(Types.TimestampType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampNanosWithoutZoneOffset.to(Types.TimestampNanoType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampNanosWithoutZoneOffset.to(Types.TimestampType.withoutZone()).value()) + .isEqualTo(1510842668000000L); + + assertThat(isoTimestampNanosWithoutZoneOffset.to(Types.TimestampNanoType.withoutZone()).value()) + .isEqualTo(1510842668000000001L); + } + + @Test + public void testTimestampMicrosWithoutZoneConversion() { + Literal<CharSequence> isoTimestampMicrosWithoutZoneOffset = + Literal.of("2017-11-16T14:31:08.000001"); + + assertThatThrownBy(() -> isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampNanoType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampType.withoutZone()).value()) + .isEqualTo(1510842668000001L); + + assertThat( + isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampNanoType.withoutZone()).value()) + .isEqualTo(1510842668000001000L); + } +} diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java index 28d01efa6d3b..fc4333d7c6c5 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -165,6 +165,62 @@ public void testLong() { .isEqualTo(hashBytes(buffer.array())); } + @Test + public void testTimestampNanoPromotion() { + Types.TimestampType tsType = Types.TimestampType.withoutZone(); + Types.TimestampNanoType tsNsType = Types.TimestampNanoType.withoutZone(); + Bucket<Object> tsNsBucket = Bucket.get(tsNsType, 1); + Bucket<Object> tsBucket = Bucket.get(tsType, 1); + + // Values from spec Appendix B: 32-bit Hash Requirements + assertThat(tsBucket.hash(Literal.of("2017-11-16T22:31:08").to(tsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + assertThat(tsNsBucket.hash(Literal.of("2017-11-16T22:31:08").to(tsNsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + + assertThat(tsBucket.hash(Literal.of("2017-11-16T22:31:08.000001").to(tsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08.000001) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + assertThat(tsNsBucket.hash(Literal.of("2017-11-16T22:31:08.000001001").to(tsNsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08.000001) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + } + + @Test + public void testTimestampTzNanoPromotion() { + Types.TimestampType tsTzType = Types.TimestampType.withZone(); + Types.TimestampNanoType tsTzNsType = Types.TimestampNanoType.withZone(); + Bucket<Object> tsTzNsBucket = Bucket.get(tsTzNsType, 1); + Bucket<Object> tsTzBucket = Bucket.get(tsTzType, 1); + + // Values from spec Appendix B: 32-bit Hash Requirements + assertThat(tsTzBucket.hash(Literal.of("2017-11-16T14:31:08-08:00").to(tsTzType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08-08:00) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + assertThat(tsTzNsBucket.hash(Literal.of("2017-11-16T14:31:08-08:00").to(tsTzNsType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08-08:00) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + + assertThat(tsTzBucket.hash(Literal.of("2017-11-16T14:31:08.000001-08:00").to(tsTzType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08.000001-08:00) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + assertThat( + tsTzNsBucket.hash( + Literal.of("2017-11-16T14:31:08.000001001-08:00").to(tsTzNsType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08.000001-08:00) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + } + @Test public void testIntegerTypePromotion() { int randomInt = testRandom.nextInt(); diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestDates.java b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java index b9c380244666..c899b4cfa1cb 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestDates.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java @@ -26,6 +26,75 @@ import org.junit.jupiter.api.Test; public class TestDates { + @Test + public void testSatisfiesOrderOfDates() { + assertThat(Dates.DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimestamps() { + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimestampNanos() { + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimeTransforms() { + assertThat(Dates.DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + @Test @SuppressWarnings("deprecation") public void testDeprecatedDateTransform() { diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java index 6101fdf0986d..93d3281411f3 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java @@ -106,7 +106,7 @@ public void testTimestampWithZoneHumanString() { // value will always be in UTC assertThat(identity.toHumanString(timestamptz, ts.value())) .as("Should produce timestamp with time zone adjusted to UTC") - .isEqualTo("2017-12-01T18:12:55.038194Z"); + .isEqualTo("2017-12-01T18:12:55.038194+00:00"); } @Test diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java new file mode 100644 index 000000000000..2a161f9bc822 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.transforms; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestTimeTransforms { + @Test + public void testMicrosSatisfiesOrderOfDates() { + assertThat(Hours.get().satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestamps() { + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestampNanos() { + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimeTransforms() { + assertThat(Hours.get().satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Years.get())).isTrue(); + } + + @Test + public void testHoursToEnum() { + Hours<Object> hours = Hours.get(); + Types.DateType type = Types.DateType.get(); + assertThatThrownBy(() -> hours.toEnum(type)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Unsupported type: date"); + } +} diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java index 3c37e643eb95..78b0e67c686b 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java @@ -26,13 +26,222 @@ import org.junit.jupiter.api.Test; public class TestTimestamps { + @Test + public void testMicrosSatisfiesOrderOfDates() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestamps() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestampNanos() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimeTransforms() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfDates() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimestamps() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimestampNanos() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimeTransforms() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + @Test @SuppressWarnings("deprecation") public void testDeprecatedTimestampTransform() { Types.TimestampType type = Types.TimestampType.withoutZone(); Literal<Long> ts = Literal.of("2017-12-01T10:12:55.038194").to(type); Literal<Long> pts = Literal.of("1970-01-01T00:00:01.000001").to(type); - Literal<Long> nts = Literal.of("1969-12-31T23:59:58.999999").to(type); + Literal<Long> nts = Literal.of("1969-12-31T23:59:59.999999").to(type); + + Transform<Long, Integer> years = Transforms.year(type); + assertThat((int) years.apply(ts.value())).as("Should produce 2017 - 1970 = 47").isEqualTo(47); + assertThat((int) years.apply(pts.value())).as("Should produce 1970 - 1970 = 0").isZero(); + assertThat((int) years.apply(nts.value())).as("Should produce 1969 - 1970 = -1").isEqualTo(-1); + + Transform<Long, Integer> months = Transforms.month(type); + assertThat((int) months.apply(ts.value())) + .as("Should produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat((int) months.apply(pts.value())).as("Should produce 0 * 12 + 0 = 0").isZero(); + assertThat((int) months.apply(nts.value())).isEqualTo(-1); + + Transform<Long, Integer> days = Transforms.day(type); + assertThat((int) days.apply(ts.value())).as("Should produce 17501").isEqualTo(17501); + assertThat((int) days.apply(pts.value())).as("Should produce 0 * 365 + 0 = 0").isZero(); + assertThat((int) days.apply(nts.value())).isEqualTo(-1); + + Transform<Long, Integer> hours = Transforms.hour(type); + assertThat((int) hours.apply(ts.value())) + .as("Should produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat((int) hours.apply(pts.value())).as("Should produce 0 * 24 + 0 = 0").isZero(); + assertThat((int) hours.apply(nts.value())).isEqualTo(-1); + } + + @Test + @SuppressWarnings("deprecation") + public void testDeprecatedTimestampNanoTransform() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal<Long> ts = Literal.of("2017-12-01T10:12:55.038194789").to(type); + Literal<Long> pts = Literal.of("1970-01-01T00:00:01.000000001").to(type); + Literal<Long> nts = Literal.of("1969-12-31T23:59:59.999999999").to(type); Transform<Long, Integer> years = Transforms.year(type); assertThat((int) years.apply(ts.value())).as("Should produce 2017 - 1970 = 47").isEqualTo(47); @@ -64,7 +273,51 @@ public void testTimestampTransform() { Types.TimestampType type = Types.TimestampType.withoutZone(); Literal<Long> ts = Literal.of("2017-12-01T10:12:55.038194").to(type); Literal<Long> pts = Literal.of("1970-01-01T00:00:01.000001").to(type); - Literal<Long> nts = Literal.of("1969-12-31T23:59:58.999999").to(type); + Literal<Long> nts = Literal.of("1969-12-31T23:59:59.999999").to(type); + + Transform<Long, Integer> years = Transforms.year(); + assertThat((int) years.bind(type).apply(ts.value())) + .as("Should produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat((int) years.bind(type).apply(pts.value())) + .as("Should produce 1970 - 1970 = 0") + .isZero(); + assertThat((int) years.bind(type).apply(nts.value())) + .as("Should produce 1969 - 1970 = -1") + .isEqualTo(-1); + + Transform<Long, Integer> months = Transforms.month(); + assertThat((int) months.bind(type).apply(ts.value())) + .as("Should produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat((int) months.bind(type).apply(pts.value())) + .as("Should produce 0 * 12 + 0 = 0") + .isZero(); + assertThat((int) months.bind(type).apply(nts.value())).isEqualTo(-1); + + Transform<Long, Integer> days = Transforms.day(); + assertThat((int) days.bind(type).apply(ts.value())).as("Should produce 17501").isEqualTo(17501); + assertThat((int) days.bind(type).apply(pts.value())) + .as("Should produce 0 * 365 + 0 = 0") + .isZero(); + assertThat((int) days.bind(type).apply(nts.value())).isEqualTo(-1); + + Transform<Long, Integer> hours = Transforms.hour(); + assertThat((int) hours.bind(type).apply(ts.value())) + .as("Should produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat((int) hours.bind(type).apply(pts.value())) + .as("Should produce 0 * 24 + 0 = 0") + .isZero(); + assertThat((int) hours.bind(type).apply(nts.value())).isEqualTo(-1); + } + + @Test + public void testTimestampNanoTransform() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal<Long> ts = Literal.of("2017-12-01T10:12:55.038194789").to(type); + Literal<Long> pts = Literal.of("1970-01-01T00:00:01.000000001").to(type); + Literal<Long> nts = Literal.of("1969-12-31T23:59:59.999999999").to(type); Transform<Long, Integer> years = Transforms.year(); assertThat((int) years.bind(type).apply(ts.value())) @@ -123,6 +376,26 @@ public void testTimestampWithoutZoneToHumanString() { .isEqualTo("2017-12-01-10"); } + @Test + public void testTimestampNanoWithoutZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal<Long> date = Literal.of("2017-12-01T10:12:55.038194789").to(type); + + Transform<Long, Integer> year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("2017"); + + Transform<Long, Integer> month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("2017-12"); + + Transform<Long, Integer> day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("2017-12-01"); + + Transform<Long, Integer> hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("2017-12-01-10"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -143,6 +416,26 @@ public void testNegativeTimestampWithoutZoneToHumanString() { .isEqualTo("1969-12-30-10"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal<Long> date = Literal.of("1969-12-30T10:12:55.038194789").to(type); + + Transform<Long, Integer> year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform<Long, Integer> month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform<Long, Integer> day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-30"); + + Transform<Long, Integer> hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-30-10"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanStringLowerBound() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -163,6 +456,26 @@ public void testNegativeTimestampWithoutZoneToHumanStringLowerBound() { .isEqualTo("1969-12-30-00"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanStringLowerBound() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal<Long> date = Literal.of("1969-12-30T00:00:00.000000000").to(type); + + Transform<Long, Integer> year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform<Long, Integer> month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform<Long, Integer> day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-30"); + + Transform<Long, Integer> hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-30-00"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanStringUpperBound() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -183,6 +496,26 @@ public void testNegativeTimestampWithoutZoneToHumanStringUpperBound() { .isEqualTo("1969-12-31-23"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanStringUpperBound() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal<Long> date = Literal.of("1969-12-31T23:59:59.999999999").to(type); + + Transform<Long, Integer> year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform<Long, Integer> month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform<Long, Integer> day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-31"); + + Transform<Long, Integer> hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-31-23"); + } + @Test public void testTimestampWithZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); @@ -205,7 +538,28 @@ public void testTimestampWithZoneToHumanString() { } @Test - public void testNullHumanString() { + public void testTimestampNanoWithZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + Literal<Long> date = Literal.of("2017-12-01T10:12:55.038194789-08:00").to(type); + + Transform<Long, Integer> year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("2017"); + + Transform<Long, Integer> month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("2017-12"); + + Transform<Long, Integer> day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("2017-12-01"); + + // the hour is 18 because the value is always UTC + Transform<Long, Integer> hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("2017-12-01-18"); + } + + @Test + public void testTimestampNullHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); assertThat(Transforms.year().toHumanString(type, null)) .as("Should produce \"null\" for null") @@ -221,6 +575,23 @@ public void testNullHumanString() { .isEqualTo("null"); } + @Test + public void testTimestampNanoNullHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + assertThat(Transforms.year().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.month().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.day().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.hour().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + } + @Test public void testTimestampsReturnType() { Types.TimestampType type = Types.TimestampType.withZone(); @@ -241,4 +612,25 @@ public void testTimestampsReturnType() { Type hourResultType = hour.getResultType(type); assertThat(hourResultType).isEqualTo(Types.IntegerType.get()); } + + @Test + public void testTimestampNanosReturnType() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + + Transform<Integer, Integer> year = Transforms.year(); + Type yearResultType = year.getResultType(type); + assertThat(yearResultType).isEqualTo(Types.IntegerType.get()); + + Transform<Integer, Integer> month = Transforms.month(); + Type monthResultType = month.getResultType(type); + assertThat(monthResultType).isEqualTo(Types.IntegerType.get()); + + Transform<Integer, Integer> day = Transforms.day(); + Type dayResultType = day.getResultType(type); + assertThat(dayResultType).isEqualTo(Types.DateType.get()); + + Transform<Integer, Integer> hour = Transforms.hour(); + Type hourResultType = hour.getResultType(type); + assertThat(hourResultType).isEqualTo(Types.IntegerType.get()); + } } diff --git a/api/src/test/java/org/apache/iceberg/types/TestComparators.java b/api/src/test/java/org/apache/iceberg/types/TestComparators.java index 165d96c029cc..07653ba3c8a8 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestComparators.java +++ b/api/src/test/java/org/apache/iceberg/types/TestComparators.java @@ -79,6 +79,12 @@ public void testTimestamp() { assertComparesCorrectly(Comparators.forType(Types.TimestampType.withZone()), 111, 222); } + @Test + public void testTimestampNanos() { + assertComparesCorrectly(Comparators.forType(Types.TimestampNanoType.withoutZone()), 111, 222); + assertComparesCorrectly(Comparators.forType(Types.TimestampNanoType.withZone()), 111, 222); + } + @Test public void testString() { assertComparesCorrectly(Comparators.forType(Types.StringType.get()), "a", "b"); diff --git a/api/src/test/java/org/apache/iceberg/types/TestConversions.java b/api/src/test/java/org/apache/iceberg/types/TestConversions.java index 6c7a884a5839..e207cfd8d59a 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestConversions.java +++ b/api/src/test/java/org/apache/iceberg/types/TestConversions.java @@ -37,6 +37,7 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampNanoType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; import org.junit.jupiter.api.Test; @@ -93,7 +94,7 @@ public void testByteBufferConversions() { assertThat(Literal.of(10000L).to(TimeType.get()).toByteBuffer().array()) .isEqualTo(new byte[] {16, 39, 0, 0, 0, 0, 0, 0}); - // timestamps are stored as microseconds from 1970-01-01 00:00:00.000000 in an 8-byte + // timestamps are stored as micro|nanoseconds from 1970-01-01 00:00:00 in an 8-byte // little-endian long // 400000L is 0...110|00011010|10000000 in binary // 10000000 -> -128, 00011010 -> 26, 00000110 -> 6, ... , 00000000 -> 0 @@ -103,6 +104,16 @@ public void testByteBufferConversions() { .isEqualTo(new byte[] {-128, 26, 6, 0, 0, 0, 0, 0}); assertThat(Literal.of(400000L).to(TimestampType.withZone()).toByteBuffer().array()) .isEqualTo(new byte[] {-128, 26, 6, 0, 0, 0, 0, 0}); + // values passed to assertConversion and Literal.of differ because Literal.of(...) assumes + // the value is in micros, which gets converted when to(TimestampNanoType) is called + assertConversion( + 400000000L, TimestampNanoType.withoutZone(), new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertConversion( + 400000000L, TimestampNanoType.withZone(), new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertThat(Literal.of(400000L).to(TimestampNanoType.withoutZone()).toByteBuffer().array()) + .isEqualTo(new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertThat(Literal.of(400000L).to(TimestampNanoType.withZone()).toByteBuffer().array()) + .isEqualTo(new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); // strings are stored as UTF-8 bytes (without length) // 'A' -> 65, 'B' -> 66, 'C' -> 67 diff --git a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java index 7f5948bd5838..2d02da5346a7 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java +++ b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java @@ -39,6 +39,8 @@ public class TestReadabilityChecks { Types.TimeType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(3), diff --git a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java index d981b5a26789..96c330d6eb43 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -41,6 +41,8 @@ public void testIdentityTypes() throws Exception { Types.TimeType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.BinaryType.get() diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypes.java b/api/src/test/java/org/apache/iceberg/types/TestTypes.java index 5db7ca7cd1bc..226c53f1e9ce 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypes.java @@ -31,6 +31,11 @@ public void fromPrimitiveString() { assertThat(Types.fromPrimitiveString("BooLean")).isSameAs(Types.BooleanType.get()); assertThat(Types.fromPrimitiveString("timestamp")).isSameAs(Types.TimestampType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamptz")).isSameAs(Types.TimestampType.withZone()); + assertThat(Types.fromPrimitiveString("timestamp_ns")) + .isSameAs(Types.TimestampNanoType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamptz_ns")) + .isSameAs(Types.TimestampNanoType.withZone()); assertThat(Types.fromPrimitiveString("Fixed[ 3 ]")).isEqualTo(Types.FixedType.ofLength(3)); diff --git a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java index 42da3a3dd774..6088fe51b57a 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java @@ -20,19 +20,72 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.time.ZonedDateTime; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; public class TestDateTimeUtil { + @Test + public void nanosToMicros() { + assertThat(DateTimeUtil.nanosToMicros(1510871468000001001L)).isEqualTo(1510871468000001L); + assertThat(DateTimeUtil.nanosToMicros(-1510871468000001001L)).isEqualTo(-1510871468000002L); + } + + @Test + public void microsToNanos() { + assertThat(DateTimeUtil.microsToNanos(1510871468000001L)).isEqualTo(1510871468000001000L); + assertThat(DateTimeUtil.microsToNanos(-1510871468000001L)).isEqualTo(-1510871468000001000L); + } + + @Test + public void isoTimestampToNanos() { + assertThat(DateTimeUtil.isoTimestampToNanos("2017-11-16T22:31:08.000001001")) + .isEqualTo(1510871468000001001L); + assertThat(DateTimeUtil.isoTimestampToNanos("1922-02-15T01:28:51.999998999")) + .isEqualTo(-1510871468000001001L); + } + + @Test + public void isoTimestamptzToNanos() { + assertThat(DateTimeUtil.isoTimestamptzToNanos("2017-11-16T14:31:08.000001001-08:00")) + .isEqualTo(1510871468000001001L); + assertThat(DateTimeUtil.isoTimestamptzToNanos("1922-02-15T01:28:51.999998999+00:00")) + .isEqualTo(-1510871468000001001L); + } + + @Test + public void convertNanos() { + assertThat( + Transforms.identity() + .toHumanString(Types.TimestampNanoType.withoutZone(), 1510871468000001001L)) + .isEqualTo("2017-11-16T22:31:08.000001001"); + assertThat(DateTimeUtil.nanosToYears(1510871468000001001L)).isEqualTo(47); + assertThat(Transforms.year().toHumanString(Types.IntegerType.get(), 47)).isEqualTo("2017"); + assertThat(DateTimeUtil.nanosToMonths(1510871468000001001L)).isEqualTo(574); + assertThat(Transforms.month().toHumanString(Types.IntegerType.get(), 574)).isEqualTo("2017-11"); + assertThat(DateTimeUtil.nanosToDays(1510871468000001001L)).isEqualTo(17486); + assertThat(Transforms.day().toHumanString(Types.IntegerType.get(), 17486)) + .isEqualTo("2017-11-16"); + assertThat(DateTimeUtil.nanosToHours(1510871468000001001L)).isEqualTo(419686); + assertThat(Transforms.hour().toHumanString(Types.IntegerType.get(), 419686)) + .isEqualTo("2017-11-16-22"); + } @Test - public void formatTimestampMillis() { - String timestamp = "1970-01-01T00:00:00.001+00:00"; - assertThat(DateTimeUtil.formatTimestampMillis(1L)).isEqualTo(timestamp); - assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1L); - - timestamp = "1970-01-01T00:16:40+00:00"; - assertThat(DateTimeUtil.formatTimestampMillis(1000000L)).isEqualTo(timestamp); - assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1000000L); + public void convertNanosNegative() { + assertThat( + Transforms.identity() + .toHumanString(Types.TimestampNanoType.withZone(), -1510871468000001001L)) + .isEqualTo("1922-02-15T01:28:51.999998999+00:00"); + assertThat(DateTimeUtil.nanosToYears(-1510871468000001001L)).isEqualTo(-48); + assertThat(Transforms.year().toHumanString(Types.IntegerType.get(), 47)).isEqualTo("2017"); + assertThat(DateTimeUtil.nanosToMonths(-1510871468000001001L)).isEqualTo(-575); + assertThat(Transforms.month().toHumanString(Types.IntegerType.get(), 574)).isEqualTo("2017-11"); + assertThat(DateTimeUtil.nanosToDays(-1510871468000001001L)).isEqualTo(-17487); + assertThat(Transforms.day().toHumanString(Types.IntegerType.get(), 17486)) + .isEqualTo("2017-11-16"); + assertThat(DateTimeUtil.nanosToHours(-1510871468000001001L)).isEqualTo(-419687); + assertThat(Transforms.hour().toHumanString(Types.IntegerType.get(), 419686)) + .isEqualTo("2017-11-16-22"); } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java index 34800cff490e..24af804b18aa 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java @@ -28,10 +28,10 @@ final class ArrowVectorAccessors { - private static final GenericArrowVectorAccessorFactory<?, String, ?, ?> factory; + private static final GenericArrowVectorAccessorFactory<?, String, ?, ?> FACTORY; static { - factory = + FACTORY = new GenericArrowVectorAccessorFactory<>( JavaDecimalFactory::new, JavaStringFactory::new, @@ -51,7 +51,7 @@ private ArrowVectorAccessors() { } static ArrowVectorAccessor<?, String, ?, ?> getVectorAccessor(VectorHolder holder) { - return factory.getVectorAccessor(holder); + return FACTORY.getVectorAccessor(holder); } private static final class JavaStringFactory implements StringFactory<String> { diff --git a/aws-bundle/LICENSE b/aws-bundle/LICENSE index a4ca14bf8f3e..0a9656c2c7c9 100644 --- a/aws-bundle/LICENSE +++ b/aws-bundle/LICENSE @@ -207,7 +207,7 @@ This binary artifact contains code from the following projects: -------------------------------------------------------------------------------- -Group: commons-codec Name: commons-codec Version: 1.15 +Group: commons-codec Name: commons-codec Version: 1.17.1 Project URL: https://commons.apache.org/proper/commons-codec/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt @@ -219,61 +219,61 @@ License: The Apache Software License, Version 2.0 - http://www.apache.org/licens -------------------------------------------------------------------------------- -Group: io.netty Name: netty-buffer Version: 4.1.86.Final +Group: io.netty Name: netty-buffer Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec Version: 4.1.86.Final +Group: io.netty Name: netty-codec Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec-http Version: 4.1.86.Final +Group: io.netty Name: netty-codec-http Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec-http2 Version: 4.1.86.Final +Group: io.netty Name: netty-codec-http2 Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-common Version: 4.1.86.Final +Group: io.netty Name: netty-common Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-handler Version: 4.1.86.Final +Group: io.netty Name: netty-handler Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-resolver Version: 4.1.86.Final +Group: io.netty Name: netty-resolver Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport Version: 4.1.86.Final +Group: io.netty Name: netty-transport Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.86.Final +Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.86.Final +Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 @@ -285,13 +285,13 @@ License: Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2. -------------------------------------------------------------------------------- -Group: org.apache.httpcomponents Name: httpcore Version: 4.4.13 +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 Project URL: http://hc.apache.org/httpcomponents-core-ga License: Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt -------------------------------------------------------------------------------- -Group: org.reactivestreams Name: reactive-streams Version: 1.0.3 +Group: org.reactivestreams Name: reactive-streams Version: 1.0.4 Project URL: http://reactive-streams.org License: CC0 - http://creativecommons.org/publicdomain/zero/1.0/ @@ -303,160 +303,220 @@ License: MIT License - http://www.opensource.org/licenses/mit-license.php -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: annotations Version: 2.20.131 +Group: software.amazon.awssdk Name: annotations Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: apache-client Version: 2.20.131 +Group: software.amazon.awssdk Name: apache-client Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: arns Version: 2.20.131 +Group: software.amazon.awssdk Name: arns Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: auth Version: 2.20.131 +Group: software.amazon.awssdk Name: auth Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-core Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-core Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: dynamodb Version: 2.20.131 +Group: software.amazon.awssdk Name: checksums Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: endpoints-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: checksums-spi Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: glue Version: 2.20.131 +Group: software.amazon.awssdk Name: crt-core Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-client-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: dynamodb Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: iam Version: 2.20.131 +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: glue Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: json-utils Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: kms Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: lakeformation Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: metrics-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: netty-nio-client Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-client-spi Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: profiles Version: 2.20.131 +Group: software.amazon.awssdk Name: iam Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: protocol-core Version: 2.20.131 +Group: software.amazon.awssdk Name: identity-spi Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: regions Version: 2.20.131 +Group: software.amazon.awssdk Name: json-utils Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: s3 Version: 2.20.131 +Group: software.amazon.awssdk Name: kms Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sdk-core Version: 2.20.131 +Group: software.amazon.awssdk Name: lakeformation Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sso Version: 2.20.131 +Group: software.amazon.awssdk Name: metrics-spi Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: profiles Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sts Version: 2.20.131 +Group: software.amazon.awssdk Name: protocol-core Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.20.131 +Group: software.amazon.awssdk Name: regions Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: s3 Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: utils Version: 2.20.131 +Group: software.amazon.awssdk Name: sdk-core Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sso Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sts Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: utils Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- +Group: software.amazon.awssdk.crt Name: aws-crt Version: 0.30.6 +Project URL: https://github.com/awslabs/aws-crt-java +License: The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: software.amazon.eventstream Name: eventstream Version: 1.0.1 Project URL: https://github.com/awslabs/aws-eventstream-java License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 diff --git a/aws-bundle/NOTICE b/aws-bundle/NOTICE index ed353940ba38..070d109c8e5f 100644 --- a/aws-bundle/NOTICE +++ b/aws-bundle/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). -------------------------------------------------------------------------------- -NOTICE for Group: commons-codec Name: commons-codec Version: 1.15 +NOTICE for Group: commons-codec Name: commons-codec Version: 1.17.1 src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains test data from http://aspell.net/test/orig/batch0.tab. @@ -23,32 +23,43 @@ Copyright (c) 2008 Alexander Beider & Stephen P. Morse. -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.20.131 +NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: checksums Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: checksums-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: crt-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: identity-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: retries Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: retries-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.27.7 AWS SDK for Java 2.0 Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. @@ -71,7 +82,7 @@ The licenses for these third party components are included in LICENSE.txt -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.20.18 +NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.27.7 # Jackson JSON processor diff --git a/aws-bundle/build.gradle b/aws-bundle/build.gradle index 6b1eae46d89e..16952d78d8e5 100644 --- a/aws-bundle/build.gradle +++ b/aws-bundle/build.gradle @@ -27,6 +27,7 @@ project(":iceberg-aws-bundle") { implementation platform(libs.awssdk.bom) implementation "software.amazon.awssdk:apache-client" implementation "software.amazon.awssdk:auth" + implementation "software.amazon.awssdk:http-auth-aws-crt" implementation "software.amazon.awssdk:iam" implementation "software.amazon.awssdk:sso" implementation "software.amazon.awssdk:s3" @@ -47,6 +48,10 @@ project(":iceberg-aws-bundle") { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // relocate AWS-specific versions relocate 'org.apache.http', 'org.apache.iceberg.aws.shaded.org.apache.http' relocate 'io.netty', 'org.apache.iceberg.aws.shaded.io.netty' diff --git a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java index bbe062d5db48..e9cf474addfa 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; @@ -93,29 +95,58 @@ public static String testAccountId() { return System.getenv("AWS_TEST_ACCOUNT_ID"); } + /** + * Set the environment variable AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS for a default account to + * use for testing. Developers need to create a S3 multi region access point before running + * integration tests because creating it takes a few minutes + * + * @return The alias of S3 multi region access point route to the default S3 bucket + */ + public static String testMultiRegionAccessPointAlias() { + return System.getenv("AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS"); + } + public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { - 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<ObjectVersion> 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<ObjectVersion> 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<String> namespaces) { for (String namespace : namespaces) { try { diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java index aa0c7f18319c..ecf589d7c07a 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueTestBase.java @@ -55,16 +55,16 @@ public class GlueTestBase { private static final Logger LOG = LoggerFactory.getLogger(GlueTestBase.class); // the integration test requires the following env variables - static final String testBucketName = AwsIntegTestUtil.testBucketName(); + static final String TEST_BUCKET_NAME = AwsIntegTestUtil.testBucketName(); - static final String catalogName = "glue"; - static final String testPathPrefix = getRandomName(); - static final List<String> namespaces = Lists.newArrayList(); + static final String CATALOG_NAME = "glue"; + static final String TEST_PATH_PREFIX = getRandomName(); + static final List<String> NAMESPACES = Lists.newArrayList(); // aws clients - static final AwsClientFactory clientFactory = AwsClientFactories.defaultFactory(); - static final GlueClient glue = clientFactory.glue(); - static final S3Client s3 = clientFactory.s3(); + static final AwsClientFactory CLIENT_FACTORY = AwsClientFactories.defaultFactory(); + static final GlueClient GLUE = CLIENT_FACTORY.glue(); + static final S3Client S3 = CLIENT_FACTORY.s3(); // iceberg static GlueCatalog glueCatalog; @@ -74,14 +74,14 @@ public class GlueTestBase { new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1")); static PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build(); // table location properties - static final Map<String, String> tableLocationProperties = + static final Map<String, String> TABLE_LOCATION_PROPERTIES = ImmutableMap.of( - TableProperties.WRITE_DATA_LOCATION, "s3://" + testBucketName + "/writeDataLoc", - TableProperties.WRITE_METADATA_LOCATION, "s3://" + testBucketName + "/writeMetaDataLoc", + TableProperties.WRITE_DATA_LOCATION, "s3://" + TEST_BUCKET_NAME + "/writeDataLoc", + TableProperties.WRITE_METADATA_LOCATION, "s3://" + TEST_BUCKET_NAME + "/writeMetaDataLoc", TableProperties.WRITE_FOLDER_STORAGE_LOCATION, - "s3://" + testBucketName + "/writeFolderStorageLoc"); + "s3://" + TEST_BUCKET_NAME + "/writeFolderStorageLoc"); - static final String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; + static final String TEST_BUCKET_PATH = "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX; @BeforeAll public static void beforeClass() { @@ -90,11 +90,11 @@ public static void beforeClass() { S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); s3FileIOProperties.setDeleteBatchSize(10); glueCatalog.initialize( - catalogName, - testBucketPath, + CATALOG_NAME, + TEST_BUCKET_PATH, awsProperties, s3FileIOProperties, - glue, + GLUE, null, ImmutableMap.of()); @@ -102,19 +102,19 @@ public static void beforeClass() { AwsProperties propertiesSkipNameValidation = new AwsProperties(); propertiesSkipNameValidation.setGlueCatalogSkipNameValidation(true); glueCatalogWithSkipNameValidation.initialize( - catalogName, - testBucketPath, + CATALOG_NAME, + TEST_BUCKET_PATH, propertiesSkipNameValidation, new S3FileIOProperties(), - glue, + GLUE, null, ImmutableMap.of()); } @AfterAll public static void afterClass() { - AwsIntegTestUtil.cleanGlueCatalog(glue, namespaces); - AwsIntegTestUtil.cleanS3Bucket(s3, testBucketName, testPathPrefix); + AwsIntegTestUtil.cleanGlueCatalog(GLUE, NAMESPACES); + AwsIntegTestUtil.cleanS3Bucket(S3, TEST_BUCKET_NAME, TEST_PATH_PREFIX); } public static String getRandomName() { @@ -123,7 +123,7 @@ public static String getRandomName() { public static String createNamespace() { String namespace = getRandomName(); - namespaces.add(namespace); + NAMESPACES.add(namespace); glueCatalog.createNamespace(Namespace.of(namespace)); return namespace; } @@ -142,7 +142,7 @@ public static String createTable(String namespace, String tableName) { public static void updateTableDescription( String namespace, String tableName, String description) { GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); Table table = response.table(); UpdateTableRequest request = UpdateTableRequest.builder() @@ -159,13 +159,13 @@ public static void updateTableDescription( .storageDescriptor(table.storageDescriptor()) .build()) .build(); - glue.updateTable(request); + GLUE.updateTable(request); } public static void updateTableColumns( String namespace, String tableName, Function<Column, Column> columnUpdater) { GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); Table existingTable = response.table(); List<Column> updatedColumns = existingTable.storageDescriptor().columns().stream() @@ -192,6 +192,6 @@ public static void updateTableColumns( .build()) .build()) .build(); - glue.updateTable(request); + GLUE.updateTable(request); } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java index f1748737874e..42b527a03742 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java @@ -510,7 +510,7 @@ private void breakFallbackCatalogCommitCheck(GlueTableOperations spyOperations) private boolean metadataFileExists(TableMetadata metadata) { try { - s3.headObject( + S3.headObject( HeadObjectRequest.builder() .bucket(S3TestUtil.getBucketFromUri(metadata.metadataFileLocation())) .key(S3TestUtil.getKeyFromUri(metadata.metadataFileLocation())) @@ -523,7 +523,7 @@ private boolean metadataFileExists(TableMetadata metadata) { private int metadataFileCount(TableMetadata metadata) { return (int) - s3 + S3 .listObjectsV2( ListObjectsV2Request.builder() .bucket(S3TestUtil.getBucketFromUri(metadata.metadataFileLocation())) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java index 53ec2a252fea..3edd9e4acdb4 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java @@ -56,18 +56,18 @@ public class TestGlueCatalogLock extends GlueTestBase { @BeforeAll public static void beforeClass() { GlueTestBase.beforeClass(); - String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; + String testBucketPath = "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX; lockTableName = getRandomName(); glueCatalog = new GlueCatalog(); AwsProperties awsProperties = new AwsProperties(); S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); - dynamo = clientFactory.dynamo(); + dynamo = CLIENT_FACTORY.dynamo(); glueCatalog.initialize( - catalogName, + CATALOG_NAME, testBucketPath, awsProperties, s3FileIOProperties, - glue, + GLUE, new DynamoDbLockManager(dynamo, lockTableName), ImmutableMap.of()); } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java index f36207005116..7a249c5509f2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java @@ -44,8 +44,8 @@ public class TestGlueCatalogNamespace extends GlueTestBase { @Test public void testCreateNamespace() { String namespace = getRandomName(); - namespaces.add(namespace); - assertThatThrownBy(() -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) + NAMESPACES.add(namespace); + assertThatThrownBy(() -> GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) .as("namespace does not exist before create") .isInstanceOf(EntityNotFoundException.class) .hasMessageContaining("not found"); @@ -60,7 +60,7 @@ public void testCreateNamespace() { Namespace ns = Namespace.of(namespace); glueCatalog.createNamespace(ns, properties); Database database = - glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.name()).isEqualTo(namespace); assertThat(database.description()).isEqualTo("description"); assertThat(database.locationUri()).isEqualTo("s3://location"); @@ -117,7 +117,7 @@ public void testNamespaceProperties() { properties.put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, "description"); glueCatalog.setProperties(Namespace.of(namespace), properties); Database database = - glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.parameters()).containsEntry("key", "val").containsEntry("key2", "val2"); assertThat(database.locationUri()).isEqualTo("s3://test"); assertThat(database.description()).isEqualTo("description"); @@ -128,7 +128,7 @@ public void testNamespaceProperties() { "key", IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, IcebergToGlueConverter.GLUE_DESCRIPTION_KEY)); - database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + database = GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.parameters()).doesNotContainKey("key").containsEntry("key2", "val2"); assertThat(database.locationUri()).isNull(); assertThat(database.description()).isNull(); @@ -138,7 +138,7 @@ public void testNamespaceProperties() { properties.put(IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, "s3://test2"); properties.put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, "description2"); glueCatalog.setProperties(Namespace.of(namespace), properties); - database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + database = GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.parameters()).containsEntry("key", "val").containsEntry("key2", "val2"); assertThat(database.locationUri()).isEqualTo("s3://test2"); assertThat(database.description()).isEqualTo("description2"); @@ -148,7 +148,7 @@ public void testNamespaceProperties() { public void testDropNamespace() { String namespace = createNamespace(); glueCatalog.dropNamespace(Namespace.of(namespace)); - assertThatThrownBy(() -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) + assertThatThrownBy(() -> GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) .as("namespace should not exist after deletion") .isInstanceOf(EntityNotFoundException.class) .hasMessageContaining("not found"); @@ -167,7 +167,7 @@ public void testDropNamespaceThatContainsOnlyIcebergTable() { @Test public void testDropNamespaceThatContainsNonIcebergTable() { String namespace = createNamespace(); - glue.createTable( + GLUE.createTable( CreateTableRequest.builder() .databaseName(namespace) .tableInput(TableInput.builder().name(UUID.randomUUID().toString()).build()) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 9c4d1839a4eb..6bd6a4ad383d 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -76,14 +76,14 @@ public void testCreateTable() { String tableDescription = "Test table"; Map<String, String> tableProperties = ImmutableMap.<String, String>builder() - .putAll(tableLocationProperties) + .putAll(TABLE_LOCATION_PROPERTIES) .put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, tableDescription) .build(); glueCatalog.createTable( TableIdentifier.of(namespace, tableName), schema, partitionSpec, tableProperties); // verify table exists in Glue GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().databaseName()).isEqualTo(namespace); assertThat(response.table().name()).isEqualTo(tableName); assertThat(response.table().parameters()) @@ -94,12 +94,12 @@ public void testCreateTable() { assertThat(response.table().storageDescriptor().columns()).hasSameSizeAs(schema.columns()); assertThat(response.table().partitionKeys()).hasSameSizeAs(partitionSpec.fields()); assertThat(response.table().storageDescriptor().additionalLocations()) - .containsExactlyInAnyOrderElementsOf(tableLocationProperties.values()); + .containsExactlyInAnyOrderElementsOf(TABLE_LOCATION_PROPERTIES.values()); // verify metadata file exists in S3 String metaLocation = response.table().parameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); - String key = metaLocation.split(testBucketName, -1)[1].substring(1); - s3.headObject(HeadObjectRequest.builder().bucket(testBucketName).key(key).build()); + String key = metaLocation.split(TEST_BUCKET_NAME, -1)[1].substring(1); + S3.headObject(HeadObjectRequest.builder().bucket(TEST_BUCKET_NAME).key(key).build()); Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); assertThat(table.spec()).isEqualTo(partitionSpec); assertThat(table.schema()).asString().isEqualTo(schema.toString()); @@ -137,18 +137,18 @@ public void testCreateTableBadName() { public void testCreateAndLoadTableWithoutWarehouseLocation() { GlueCatalog glueCatalogWithoutWarehouse = new GlueCatalog(); glueCatalogWithoutWarehouse.initialize( - catalogName, + CATALOG_NAME, null, new AwsProperties(), new S3FileIOProperties(), - glue, + GLUE, LockManagers.defaultLockManager(), ImmutableMap.of()); String namespace = createNamespace(); String tableName = getRandomName(); TableIdentifier identifier = TableIdentifier.of(namespace, tableName); try { - glueCatalog.createTable(identifier, schema, partitionSpec, tableLocationProperties); + glueCatalog.createTable(identifier, schema, partitionSpec, TABLE_LOCATION_PROPERTIES); glueCatalog.loadTable(identifier); } catch (RuntimeException e) { throw new RuntimeException( @@ -202,7 +202,7 @@ public void testUpdateTable() { assertThat(table.history()).hasSize(1); // check table in Glue GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().tableType()) .as("external table type is set after update") .isEqualTo("EXTERNAL_TABLE"); @@ -217,7 +217,7 @@ public void testUpdateTable() { .commit(); // check table in Glue response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().description()).isEqualTo(updatedComment); } @@ -246,7 +246,7 @@ public void testDropColumn() { table.updateSchema().deleteColumn("c2").deleteColumn("c3").commit(); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); List<Column> actualColumns = response.table().storageDescriptor().columns(); List<Column> expectedColumns = @@ -308,7 +308,7 @@ public void testRenameTableFailsToCreateNewTable() { Table table = glueCatalog.loadTable(id); // create a new table in Glue, so that rename to that table will fail String newTableName = tableName + "_2"; - glue.createTable( + GLUE.createTable( CreateTableRequest.builder() .databaseName(namespace) .tableInput(TableInput.builder().name(newTableName).build()) @@ -337,7 +337,7 @@ public void testRenameTableFailsToDeleteOldTable() { Table table = glueCatalog.loadTable(id); // delete the old table metadata, so that drop old table will fail String newTableName = tableName + "_2"; - glue.updateTable( + GLUE.updateTable( UpdateTableRequest.builder() .databaseName(namespace) .tableInput(TableInput.builder().name(tableName).parameters(Maps.newHashMap()).build()) @@ -352,7 +352,7 @@ public void testRenameTableFailsToDeleteOldTable() { .hasMessageContaining("Input Glue table is not an iceberg table"); assertThatThrownBy( () -> - glue.getTable( + GLUE.getTable( GetTableRequest.builder().databaseName(namespace).name(newTableName).build())) .isInstanceOf(EntityNotFoundException.class) .as("renamed table should be deleted") @@ -370,11 +370,11 @@ public void testDeleteTableWithoutPurge() { .hasMessageContaining("Table does not exist"); String warehouseLocation = glueCatalog.defaultWarehouseLocation(TableIdentifier.of(namespace, tableName)); - String prefix = warehouseLocation.split(testBucketName + "/", -1)[1]; + String prefix = warehouseLocation.split(TEST_BUCKET_NAME + "/", -1)[1]; ListObjectsV2Response response = - s3.listObjectsV2( + S3.listObjectsV2( ListObjectsV2Request.builder() - .bucket(testBucketName) + .bucket(TEST_BUCKET_NAME) .prefix(prefix + "/metadata/") .build()); assertThat(response.hasContents()).isTrue(); @@ -423,10 +423,10 @@ public void testDeleteTableWithPurge() { .hasMessageContaining("Table does not exist"); String warehouseLocation = glueCatalog.defaultWarehouseLocation(TableIdentifier.of(namespace, tableName)); - String prefix = warehouseLocation.split(testBucketName + "/", -1)[1]; + String prefix = warehouseLocation.split(TEST_BUCKET_NAME + "/", -1)[1]; ListObjectsV2Response response = - s3.listObjectsV2( - ListObjectsV2Request.builder().bucket(testBucketName).prefix(prefix).build()); + S3.listObjectsV2( + ListObjectsV2Request.builder().bucket(TEST_BUCKET_NAME).prefix(prefix).build()); if (response.hasContents()) { // might have directory markers left for (S3Object s3Object : response.contents()) { @@ -441,7 +441,7 @@ public void testDeleteTableWithPurge() { public void testCommitTableSkipArchive() { // create ns String namespace = getRandomName(); - namespaces.add(namespace); + NAMESPACES.add(namespace); glueCatalog.createNamespace(Namespace.of(namespace)); // create table and commit without skip Schema schema = new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1")); @@ -450,11 +450,11 @@ public void testCommitTableSkipArchive() { AwsProperties properties = new AwsProperties(); properties.setGlueCatalogSkipArchive(false); glueCatalog.initialize( - catalogName, - testBucketPath, + CATALOG_NAME, + TEST_BUCKET_PATH, properties, new S3FileIOProperties(), - glue, + GLUE, LockManagers.defaultLockManager(), ImmutableMap.of()); glueCatalog.createTable(TableIdentifier.of(namespace, tableName), schema, partitionSpec); @@ -467,7 +467,7 @@ public void testCommitTableSkipArchive() { .build(); table.newAppend().appendFile(dataFile).commit(); assertThat( - glue.getTableVersions( + GLUE.getTableVersions( GetTableVersionsRequest.builder() .databaseName(namespace) .tableName(tableName) @@ -476,12 +476,12 @@ public void testCommitTableSkipArchive() { .hasSize(2); // create table and commit with skip tableName = getRandomName(); - glueCatalog.initialize(catalogName, ImmutableMap.of()); + glueCatalog.initialize(CATALOG_NAME, ImmutableMap.of()); glueCatalog.createTable(TableIdentifier.of(namespace, tableName), schema, partitionSpec); table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); table.newAppend().appendFile(dataFile).commit(); assertThat( - glue.getTableVersions( + GLUE.getTableVersions( GetTableVersionsRequest.builder() .databaseName(namespace) .tableName(tableName) @@ -494,13 +494,13 @@ public void testCommitTableSkipArchive() { @Test public void testCommitTableSkipNameValidation() { String namespace = "dd-dd"; - namespaces.add(namespace); + NAMESPACES.add(namespace); glueCatalogWithSkipNameValidation.createNamespace(Namespace.of(namespace)); String tableName = "cc-cc"; glueCatalogWithSkipNameValidation.createTable( - TableIdentifier.of(namespace, tableName), schema, partitionSpec, tableLocationProperties); + TableIdentifier.of(namespace, tableName), schema, partitionSpec, TABLE_LOCATION_PROPERTIES); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().databaseName()).isEqualTo(namespace); assertThat(response.table().name()).isEqualTo(tableName); } @@ -522,7 +522,7 @@ public void testColumnCommentsAndParameters() { table.updateSpec().addField(truncate("c1", 8)).commit(); table.updateSchema().deleteColumn("c3").renameColumn("c4", "c5").commit(); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); List<Column> actualColumns = response.table().storageDescriptor().columns(); List<Column> expectedColumns = @@ -605,7 +605,7 @@ public void testGlueTableColumnCommentsPreserved() { .commit(); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); List<Column> actualColumns = response.table().storageDescriptor().columns(); List<Column> expectedColumns = @@ -655,7 +655,7 @@ public void testTablePropsDefinedAtCatalogLevel() { "table-default.key3", "catalog-default-key3", "table-override.key3", "catalog-override-key3", "table-override.key4", "catalog-override-key4", - "warehouse", "s3://" + testBucketName + "/" + testPathPrefix); + "warehouse", "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX); glueCatalog.initialize("glue", catalogProps); @@ -722,7 +722,7 @@ public void testRegisterTableAlreadyExists() { @Test public void testTableLevelS3Tags() { - String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; + String testBucketPath = "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX; Map<String, String> properties = ImmutableMap.of( S3FileIOProperties.WRITE_TABLE_TAG_ENABLED, @@ -730,11 +730,11 @@ public void testTableLevelS3Tags() { S3FileIOProperties.WRITE_NAMESPACE_TAG_ENABLED, "true"); glueCatalog.initialize( - catalogName, + CATALOG_NAME, testBucketPath, new AwsProperties(properties), new S3FileIOProperties(properties), - glue, + GLUE, null); String namespace = createNamespace(); String tableName = getRandomName(); @@ -742,13 +742,13 @@ public void testTableLevelS3Tags() { // Get metadata object tag from S3 GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); String metaLocation = response.table().parameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); - String key = metaLocation.split(testBucketName, -1)[1].substring(1); + String key = metaLocation.split(TEST_BUCKET_NAME, -1)[1].substring(1); List<Tag> tags = - s3.getObjectTagging( - GetObjectTaggingRequest.builder().bucket(testBucketName).key(key).build()) + S3.getObjectTagging( + GetObjectTaggingRequest.builder().bucket(TEST_BUCKET_NAME).key(key).build()) .tagSet(); Map<String, String> tagMap = tags.stream().collect(Collectors.toMap(Tag::key, Tag::value)); 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..fc2b9f97f1e6 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/s3/TestS3FileIOIntegration.java @@ -41,6 +41,7 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.assertj.core.api.Assumptions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -53,14 +54,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; @@ -72,11 +76,13 @@ public class TestS3FileIOIntegration { private static S3Client s3; private static S3ControlClient s3Control; private static S3ControlClient crossRegionS3Control; + private static S3ControlClient multiRegionS3Control; private static KmsClient kms; private static String bucketName; private static String crossRegionBucketName; private static String accessPointName; private static String crossRegionAccessPointName; + private static String multiRegionAccessPointAlias; private static String prefix; private static byte[] contentBytes; private static String content; @@ -106,6 +112,13 @@ public static void beforeClass() { AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); AwsIntegTestUtil.createAccessPoint( crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); + multiRegionAccessPointAlias = AwsIntegTestUtil.testMultiRegionAccessPointAlias(); + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) + .build()); } @AfterAll @@ -193,15 +206,32 @@ public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception { validateRead(s3FileIO); } + @Test + public void testNewInputStreamWithMultiRegionAccessPoint() throws Exception { + Assumptions.assumeThat(multiRegionAccessPointAlias).isNotEmpty(); + clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); + S3Client s3Client = clientFactory.s3(); + s3Client.putObject( + PutObjectRequest.builder().bucket(bucketName).key(objectKey).build(), + RequestBody.fromBytes(contentBytes)); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + s3FileIO.initialize( + ImmutableMap.of( + S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, + testMultiRegionAccessPointARN( + AwsIntegTestUtil.testRegion(), multiRegionAccessPointAlias))); + validateRead(s3FileIO); + } + @Test public void testNewOutputStream() throws Exception { S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); write(s3FileIO); - InputStream stream = - s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -212,11 +242,11 @@ public void testNewOutputStreamWithAccessPoint() throws Exception { S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, testAccessPointARN(AwsIntegTestUtil.testRegion(), accessPointName))); write(s3FileIO); - InputStream stream = - s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -229,17 +259,35 @@ public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception { S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, testAccessPointARN(AwsIntegTestUtil.testCrossRegion(), crossRegionAccessPointName))); write(s3FileIO); - InputStream stream = + try (InputStream stream = s3Client.getObject( GetObjectRequest.builder() .bucket( testAccessPointARN( AwsIntegTestUtil.testCrossRegion(), crossRegionAccessPointName)) .key(objectKey) - .build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + .build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } + } + + @Test + public void testNewOutputStreamWithMultiRegionAccessPoint() throws Exception { + Assumptions.assumeThat(multiRegionAccessPointAlias).isNotEmpty(); + clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + s3FileIO.initialize( + ImmutableMap.of( + S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, + testMultiRegionAccessPointARN( + AwsIntegTestUtil.testRegion(), multiRegionAccessPointAlias))); + write(s3FileIO); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -445,6 +493,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); @@ -470,18 +547,18 @@ private void write(S3FileIO s3FileIO) throws Exception { private void write(S3FileIO s3FileIO, String uri) throws Exception { OutputFile outputFile = s3FileIO.newOutputFile(uri); - OutputStream outputStream = outputFile.create(); - IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream); - outputStream.close(); + try (OutputStream outputStream = outputFile.create()) { + IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream); + } } private void validateRead(S3FileIO s3FileIO) throws Exception { InputFile file = s3FileIO.newInputFile(objectUri); assertThat(file.getLength()).isEqualTo(contentBytes.length); - InputStream stream = file.newStream(); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = file.newStream()) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } private String testAccessPointARN(String region, String accessPoint) { @@ -494,6 +571,13 @@ private String testAccessPointARN(String region, String accessPoint) { accessPoint); } + private String testMultiRegionAccessPointARN(String region, String alias) { + // format: arn:aws:s3::account-id:accesspoint/MultiRegionAccessPoint_alias + return String.format( + "arn:%s:s3::%s:accesspoint/%s", + PartitionMetadata.of(Region.of(region)).id(), AwsIntegTestUtil.testAccountId(), alias); + } + private void createRandomObjects(String objectPrefix, int count) { S3URI s3URI = new S3URI(objectPrefix); random diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index c6b157bb5c79..47807a2b9f37 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.aws.glue; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.RemovalListener; import java.io.IOException; import java.util.List; import java.util.Map; @@ -51,7 +48,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; -import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOTracker; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; @@ -97,7 +94,7 @@ public class GlueCatalog extends BaseMetastoreCatalog private LockManager lockManager; private CloseableGroup closeableGroup; private Map<String, String> catalogProperties; - private Cache<TableOperations, FileIO> fileIOCloser; + private FileIOTracker fileIOTracker; // Attempt to set versionId if available on the path private static final DynMethods.UnboundMethod SET_VERSION_ID = @@ -194,11 +191,12 @@ void initialize( this.lockManager = lock; this.closeableGroup = new CloseableGroup(); + this.fileIOTracker = new FileIOTracker(); closeableGroup.addCloseable(glue); closeableGroup.addCloseable(lockManager); closeableGroup.addCloseable(metricsReporter()); + closeableGroup.addCloseable(fileIOTracker); closeableGroup.setSuppressCloseFailure(true); - this.fileIOCloser = newFileIOCloser(); } @Override @@ -243,7 +241,7 @@ protected TableOperations newTableOps(TableIdentifier tableIdentifier) { tableSpecificCatalogPropertiesBuilder.buildOrThrow(), hadoopConf, tableIdentifier); - fileIOCloser.put(glueTableOperations, glueTableOperations.io()); + fileIOTracker.track(glueTableOperations); return glueTableOperations; } @@ -256,7 +254,7 @@ protected TableOperations newTableOps(TableIdentifier tableIdentifier) { catalogProperties, hadoopConf, tableIdentifier); - fileIOCloser.put(glueTableOperations, glueTableOperations.io()); + fileIOTracker.track(glueTableOperations); return glueTableOperations; } @@ -634,10 +632,6 @@ public String name() { @Override public void close() throws IOException { closeableGroup.close(); - if (fileIOCloser != null) { - fileIOCloser.invalidateAll(); - fileIOCloser.cleanUp(); - } } @Override @@ -649,17 +643,4 @@ public void setConf(Configuration conf) { protected Map<String, String> properties() { return catalogProperties == null ? ImmutableMap.of() : catalogProperties; } - - private Cache<TableOperations, FileIO> newFileIOCloser() { - return Caffeine.newBuilder() - .weakKeys() - .removalListener( - (RemovalListener<TableOperations, FileIO>) - (ops, fileIO, cause) -> { - if (null != fileIO) { - fileIO.close(); - } - }) - .build(); - } } 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..040d3012ca7a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java @@ -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"; @@ -406,7 +412,7 @@ public void close() { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); @@ -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<ObjectVersion> 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/main/java/org/apache/iceberg/aws/s3/S3InputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java index f1d6c30a27a5..f442a0f04a1c 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java @@ -227,7 +227,7 @@ public void setSkipSize(int skipSize) { this.skipSize = skipSize; } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java index 046abdb61e13..4e35c77d0557 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java @@ -76,7 +76,7 @@ class S3OutputStream extends PositionOutputStream { private static final Logger LOG = LoggerFactory.getLogger(S3OutputStream.class); - private static final String digestAlgorithm = "MD5"; + private static final String DIGEST_ALGORITHM = "MD5"; private static volatile ExecutorService executorService; @@ -138,7 +138,7 @@ class S3OutputStream extends PositionOutputStream { this.isChecksumEnabled = s3FileIOProperties.isChecksumEnabled(); try { this.completeMessageDigest = - isChecksumEnabled ? MessageDigest.getInstance(digestAlgorithm) : null; + isChecksumEnabled ? MessageDigest.getInstance(DIGEST_ALGORITHM) : null; } catch (NoSuchAlgorithmException e) { throw new RuntimeException( "Failed to create message digest needed for s3 checksum checks", e); @@ -220,7 +220,7 @@ private void newStream() throws IOException { currentStagingFile.deleteOnExit(); try { currentPartMessageDigest = - isChecksumEnabled ? MessageDigest.getInstance(digestAlgorithm) : null; + isChecksumEnabled ? MessageDigest.getInstance(DIGEST_ALGORITHM) : null; } catch (NoSuchAlgorithmException e) { throw new RuntimeException( "Failed to create message digest needed for s3 checksum checks.", e); @@ -253,6 +253,10 @@ private void newStream() throws IOException { @Override public void close() throws IOException { + close(true); + } + + private void close(boolean completeUploads) throws IOException { if (closed) { return; } @@ -262,7 +266,9 @@ public void close() throws IOException { try { stream.close(); - completeUploads(); + if (completeUploads) { + completeUploads(); + } } finally { cleanUpStagingFiles(); } @@ -475,12 +481,12 @@ private void createStagingDirectoryIfNotExists() throws IOException, SecurityExc } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); if (!closed) { - close(); // releasing resources is more important than printing the warning + close(false); // releasing resources is more important than printing the warning String trace = Joiner.on("\n\t").join(Arrays.copyOfRange(createStack, 1, createStack.length)); LOG.warn("Unclosed output stream created by:\n\t{}", trace); } 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..1cc34a59cc1b 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java @@ -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; @@ -383,6 +384,18 @@ public void testResolvingFileIOLoad() { assertThat(result).isInstanceOf(S3FileIO.class); } + @Test + public void testResolvingFileIOLoadWithoutConf() { + ResolvingFileIO resolvingFileIO = new ResolvingFileIO(); + resolvingFileIO.initialize(ImmutableMap.of()); + FileIO result = + DynMethods.builder("io") + .hiddenImpl(ResolvingFileIO.class, String.class) + .build(resolvingFileIO) + .invoke("s3://foo/bar"); + assertThat(result).isInstanceOf(S3FileIO.class); + } + @Test public void testInputFileWithDataFile() throws IOException { String location = "s3://bucket/path/to/data-file.parquet"; @@ -447,7 +460,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<String, String> 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<String, String> 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..f2a70aee29e5 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3RestSigner.java @@ -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") @@ -165,12 +171,8 @@ private static Server initHttpServer() throws Exception { new S3SignerServlet(S3ObjectMapper.mapper(), ImmutableList.of(deleteObjectsWithBody)); ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); - servletContext.setContextPath("/"); - ServletHolder servletHolder = new ServletHolder(servlet); - servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); - servletContext.addServlet(servletHolder, "/*"); - servletContext.setVirtualHosts(null); - servletContext.setGzipHandler(new GzipHandler()); + servletContext.addServlet(new ServletHolder(servlet), "/*"); + servletContext.setHandler(new GzipHandler()); Server server = new Server(0); server.setHandler(servletContext); diff --git a/azure-bundle/build.gradle b/azure-bundle/build.gradle index 3bc0a31cb46d..9b1cd5b42fb8 100644 --- a/azure-bundle/build.gradle +++ b/azure-bundle/build.gradle @@ -39,6 +39,10 @@ project(":iceberg-azure-bundle") { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // relocate Azure-specific versions relocate 'io.netty', 'org.apache.iceberg.azure.shaded.io.netty' relocate 'com.fasterxml.jackson', 'org.apache.iceberg.azure.shaded.com.fasterxml.jackson' diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java index 261ede62db15..d9e1f93bea67 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java @@ -190,7 +190,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java index f38a408896be..b507d5b54115 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java @@ -110,7 +110,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/baseline.gradle b/baseline.gradle index c890ada8de57..be2dc1198e94 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -42,10 +42,24 @@ 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' - apply plugin: 'com.diffplug.spotless' + // We need to update Google Java Format to 1.17.0+ to run spotless on JDK 8, but that requires dropping support for JDK 8. + if (JavaVersion.current() == JavaVersion.VERSION_21) { + task spotlessApply { + doLast { + throw new GradleException("Spotless plugin is currently disabled when running on JDK 21 (until we drop JDK 8). To run spotlessApply please use a different JDK version.") + } + } + } else { + apply plugin: 'com.diffplug.spotless' + } pluginManager.withPlugin('com.palantir.baseline-checkstyle') { checkstyle { @@ -59,7 +73,7 @@ subprojects { pluginManager.withPlugin('com.diffplug.spotless') { spotless { java { - target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' + target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/testFixtures/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' // we use an older version of google-java-format that is compatible with JDK 8 googleJavaFormat("1.7") removeUnusedImports() @@ -77,11 +91,16 @@ subprojects { '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', + '-Xep:DangerousJavaDeserialization:ERROR', '-Xep:DangerousThreadPoolExecutorUsage:OFF', + '-Xep:DefaultCharset:ERROR', + '-Xep:DefaultLocale:ERROR', // subclasses are not equal '-Xep:EqualsGetClass:OFF', // specific to Palantir '-Xep:FinalClass:OFF', + '-Xep:FormatStringAnnotation:ERROR', + '-Xep:ImmutablesReferenceEquality:ERROR', '-Xep:IntLongMath:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', @@ -94,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', @@ -111,6 +132,9 @@ subprojects { '-Xep:StringSplitter:ERROR', '-Xep:TypeParameterShadowing:OFF', '-Xep:TypeParameterUnusedInFormals:OFF', + // Palantir's UnnecessarilyQualified may throw during analysis + '-Xep:UnnecessarilyQualified:OFF', + '-Xep:UnusedMethod:ERROR', ) } } diff --git a/build.gradle b/build.gradle index 1cb28078d233..6857a8443ef7 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,14 +51,11 @@ 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) { - project.ext.jdkVersion = '17' +} else if (JavaVersion.current() == JavaVersion.VERSION_17 || JavaVersion.current() == JavaVersion.VERSION_21) { + project.ext.jdkVersion = JavaVersion.current().getMajorVersion().toString() project.ext.extraJvmArgs = ["--add-opens", "java.base/java.io=ALL-UNNAMED", "--add-opens", "java.base/java.lang.invoke=ALL-UNNAMED", "--add-opens", "java.base/java.lang.reflect=ALL-UNNAMED", @@ -86,7 +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 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 { @@ -143,7 +135,7 @@ subprojects { revapi { oldGroup = project.group oldName = project.name - oldVersion = "1.5.0" + oldVersion = "1.6.0" } tasks.register('showDeprecationRulesOnRevApiFailure') { @@ -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 @@ -480,6 +471,7 @@ project(':iceberg-aws') { compileOnly("software.amazon.awssdk:url-connection-client") compileOnly("software.amazon.awssdk:apache-client") compileOnly("software.amazon.awssdk:auth") + compileOnly("software.amazon.awssdk:http-auth-aws-crt") compileOnly("software.amazon.awssdk:s3") compileOnly("software.amazon.awssdk:kms") compileOnly("software.amazon.awssdk:glue") @@ -887,23 +879,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 +899,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 } } @@ -978,6 +953,51 @@ project(':iceberg-snowflake') { } project(':iceberg-open-api') { + apply plugin: 'java-test-fixtures' + + dependencies { + testImplementation project(':iceberg-api') + testImplementation project(':iceberg-core') + testImplementation project(':iceberg-core').sourceSets.test.runtimeClasspath + testImplementation(testFixtures(project(':iceberg-open-api'))) + + testImplementation libs.junit.jupiter + testImplementation libs.junit.suite.api + testImplementation libs.junit.suite.engine + testImplementation libs.assertj.core + + testImplementation project(':iceberg-aws-bundle') + testImplementation project(':iceberg-gcp-bundle') + testImplementation project(':iceberg-azure-bundle') + + testFixturesImplementation project(':iceberg-api') + testFixturesImplementation project(':iceberg-core') + testFixturesImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testFixturesImplementation project(':iceberg-core').sourceSets.test.runtimeClasspath + testFixturesImplementation project(':iceberg-aws') + testFixturesImplementation project(':iceberg-gcp') + testFixturesImplementation project(':iceberg-azure') + + testFixturesImplementation libs.jetty.servlet + testFixturesImplementation libs.jetty.server + testFixturesImplementation libs.sqlite.jdbc + } + + test { + useJUnitPlatform() + + // Always rerun the compatibility tests + outputs.upToDateWhen {false} + maxParallelForks = 1 + + // Pass through any system properties that start with "rck" (REST Compatibility Kit) + // Note: only pass through specific properties so they do not affect other build/test + // configurations + systemProperties System.properties + .findAll { k, v -> k.startsWith("rck") } + .collectEntries { k, v -> { [(k):v, (k.replaceFirst("rck.", "")):v] }} // strip prefix + } + def restCatalogSpec = "$projectDir/rest-catalog-open-api.yaml" tasks.register('validateRESTCatalogSpec', org.openapitools.generator.gradle.plugin.tasks.ValidateTask) { inputSpec.set(restCatalogSpec) diff --git a/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<C> constructor, Class<? extends C> constructed) { this.constructed = constructed; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Class<? extends C> getConstructedClass() { - return constructed; - } - public C newInstanceChecked(Object... args) throws Exception { try { if (args.length > ctor.getParameterCount()) { @@ -82,6 +76,8 @@ public <R> 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> R invokeChecked(Object target, Object... args) throws Exception { @@ -172,16 +168,6 @@ public <T> Builder impl(Class<T> 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 a96da851315b..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,19 +386,6 @@ public <T> BoundField<T> build(Object target) { return this.<T>build().bind(target); } - /** - * Returns the first valid implementation as a StaticField or throws a NoSuchFieldException if - * there is none. - * - * @param <T> 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 - */ - public <T> StaticField<T> buildStaticChecked() throws NoSuchFieldException { - return this.<T>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 98de1e7f06ca..65a69bd0e12c 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -52,7 +52,7 @@ public static class UnboundMethod { } @SuppressWarnings("unchecked") - public <R> R invokeChecked(Object target, Object... args) throws Exception { + <R> R invokeChecked(Object target, Object... args) throws Exception { try { if (argLength < 0) { return (R) method.invoke(target, args); @@ -125,8 +125,10 @@ 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> R invokeChecked(Object target, Object... args) throws Exception { + public <R> R invokeChecked(Object target, Object... args) { return null; } @@ -313,34 +315,6 @@ public Builder impl(Class<?> targetClass, Class<?>... argClasses) { return this; } - 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; - } - - 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/jmh/java/org/apache/iceberg/AppendBenchmark.java b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java new file mode 100644 index 000000000000..a8bafe413c6f --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the performance of appending files to the table. + * + * <p>To run this benchmark: <code> + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=AppendBenchmark + * -PjmhOutputPath=benchmark/append-benchmark.txt + * </code> + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 10, timeUnit = TimeUnit.MINUTES) +public class AppendBenchmark { + + private static final String TABLE_IDENT = "tbl"; + private static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final HadoopTables TABLES = new HadoopTables(); + + private Table table; + private List<DataFile> dataFiles; + + @Param({"500000", "1000000", "2500000"}) + private int numFiles; + + @Param({"true", "false"}) + private boolean fast; + + @Setup + public void setupBenchmark() { + initTable(); + initDataFiles(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + } + + @Benchmark + @Threads(1) + public void appendFiles(Blackhole blackhole) { + AppendFiles append = fast ? table.newFastAppend() : table.newAppend(); + + for (DataFile dataFile : dataFiles) { + append.appendFile(dataFile); + } + + append.commit(); + } + + private void initTable() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + } + + private void dropTable() { + TABLES.dropTable(TABLE_IDENT); + } + + private void initDataFiles() { + List<DataFile> generatedDataFiles = Lists.newArrayListWithExpectedSize(numFiles); + + for (int ordinal = 0; ordinal < numFiles; ordinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, null); + generatedDataFiles.add(dataFile); + } + + this.dataFiles = generatedDataFiles; + } +} diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java index 6677e5d8b651..e4a63beceaa6 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java @@ -69,10 +69,9 @@ public void before() { manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); Random random = new Random(System.currentTimeMillis()); - ManifestListWriter listWriter = - ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0); - try { + try (ManifestListWriter listWriter = + ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0)) { for (int i = 0; i < NUM_FILES; i++) { OutputFile manifestFile = org.apache.iceberg.Files.localOutput( @@ -98,8 +97,6 @@ public void before() { listWriter.add(writer.toManifestFile()); } - - listWriter.close(); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java new file mode 100644 index 000000000000..c175248241e9 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.io.Files; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; + +/** + * A benchmark that evaluates the performance of writing manifest files + * + * <p>To run this benchmark: <code> + * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestWriteBenchmark + * </code> + */ +@Fork(1) +@State(Scope.Benchmark) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 5, timeUnit = TimeUnit.MINUTES) +public class ManifestWriteBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 100000; + private static final int NUM_COLS = 100; + + private String baseDir; + private String manifestListFile; + + private Metrics metrics; + + @Setup + public void before() { + Random random = new Random(System.currentTimeMillis()); + // Pre-create the metrics to avoid doing this in the benchmark itself + metrics = randomMetrics(random); + } + + @TearDown + public void after() { + if (baseDir != null) { + FileUtils.deleteQuietly(new File(baseDir)); + baseDir = null; + } + + manifestListFile = null; + } + + @State(Scope.Benchmark) + public static class BenchmarkState { + @Param({"1", "2"}) + private int formatVersion; + + public int getFormatVersion() { + return formatVersion; + } + } + + @Benchmark + @Threads(1) + public void writeManifestFile(BenchmarkState state) throws IOException { + this.baseDir = Files.createTempDir().getAbsolutePath(); + this.manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); + + try (ManifestListWriter listWriter = + ManifestLists.write( + state.getFormatVersion(), + org.apache.iceberg.Files.localOutput(manifestListFile), + 0, + 1L, + 0)) { + for (int i = 0; i < NUM_FILES; i++) { + OutputFile manifestFile = + org.apache.iceberg.Files.localOutput( + String.format("%s/%s.avro", baseDir, UUID.randomUUID())); + + ManifestWriter<DataFile> writer = + ManifestFiles.write( + state.formatVersion, PartitionSpec.unpartitioned(), manifestFile, 1L); + try (ManifestWriter<DataFile> finalWriter = writer) { + for (int j = 0; j < NUM_ROWS; j++) { + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFormat(FileFormat.PARQUET) + .withPath(String.format("/path/to/data-%s-%s.parquet", i, j)) + .withFileSizeInBytes(j) + .withRecordCount(j) + .withMetrics(metrics) + .build(); + finalWriter.add(dataFile); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + listWriter.add(writer.toManifestFile()); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private Metrics randomMetrics(Random random) { + long rowCount = 100000L + random.nextInt(1000); + Map<Integer, Long> columnSizes = Maps.newHashMap(); + Map<Integer, Long> valueCounts = Maps.newHashMap(); + Map<Integer, Long> nullValueCounts = Maps.newHashMap(); + Map<Integer, Long> nanValueCounts = Maps.newHashMap(); + Map<Integer, ByteBuffer> lowerBounds = Maps.newHashMap(); + Map<Integer, ByteBuffer> upperBounds = Maps.newHashMap(); + for (int i = 0; i < NUM_COLS; i++) { + columnSizes.put(i, 1000000L + random.nextInt(100000)); + valueCounts.put(i, 100000L + random.nextInt(100)); + nullValueCounts.put(i, (long) random.nextInt(5)); + nanValueCounts.put(i, (long) random.nextInt(5)); + byte[] lower = new byte[8]; + random.nextBytes(lower); + lowerBounds.put(i, ByteBuffer.wrap(lower)); + byte[] upper = new byte[8]; + random.nextBytes(upper); + upperBounds.put(i, ByteBuffer.wrap(upper)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } +} diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java index 4a85ab3ce5ed..037dba19e970 100644 --- a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java @@ -34,6 +34,7 @@ import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -52,7 +53,8 @@ public class AllManifestsTable extends BaseMetadataTable { public static final Types.NestedField REF_SNAPSHOT_ID = Types.NestedField.required(18, "reference_snapshot_id", Types.LongType.get()); - private static final Schema MANIFEST_FILE_SCHEMA = + @VisibleForTesting + static final Schema MANIFEST_FILE_SCHEMA = new Schema( Types.NestedField.required(14, "content", Types.IntegerType.get()), Types.NestedField.required(1, "path", Types.StringType.get()), @@ -119,6 +121,7 @@ protected TableScan newRefinedScan(Table table, Schema schema, TableScanContext protected CloseableIterable<FileScanTask> doPlanFiles() { FileIO io = table().io(); Map<Integer, PartitionSpec> specs = Maps.newHashMap(table().specs()); + Schema dataTableSchema = table().schema(); Expression filter = shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter(); SnapshotEvaluator snapshotEvaluator = @@ -132,7 +135,13 @@ protected CloseableIterable<FileScanTask> doPlanFiles() { snap -> { if (snap.manifestListLocation() != null) { return new ManifestListReadTask( - io, schema(), specs, snap.manifestListLocation(), filter, snap.snapshotId()); + dataTableSchema, + io, + schema(), + specs, + snap.manifestListLocation(), + filter, + snap.snapshotId()); } else { return StaticDataTask.of( io.newInputFile( @@ -149,6 +158,7 @@ protected CloseableIterable<FileScanTask> doPlanFiles() { } static class ManifestListReadTask implements DataTask { + private final Schema dataTableSchema; private final FileIO io; private final Schema schema; private final Map<Integer, PartitionSpec> specs; @@ -158,12 +168,14 @@ static class ManifestListReadTask implements DataTask { private DataFile lazyDataFile = null; ManifestListReadTask( + Schema dataTableSchema, FileIO io, Schema schema, Map<Integer, PartitionSpec> specs, String manifestListLocation, Expression residual, long referenceSnapshotId) { + this.dataTableSchema = dataTableSchema; this.io = io; this.schema = schema; this.specs = specs; @@ -244,6 +256,31 @@ public Expression residual() { public Iterable<FileScanTask> split(long splitSize) { return ImmutableList.of(this); // don't split } + + @Override + public Schema schema() { + return schema; + } + + Schema dataTableSchema() { + return dataTableSchema; + } + + FileIO io() { + return io; + } + + Map<Integer, PartitionSpec> specsById() { + return specs; + } + + String manifestListLocation() { + return manifestListLocation; + } + + long referenceSnapshotId() { + return referenceSnapshotId; + } } static StaticDataTask.Row manifestFileToRow( diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java b/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java new file mode 100644 index 000000000000..24c28d95466c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; + +class AllManifestsTableTaskParser { + private static final String DATA_TABLE_SCHEMA = "data-table-schema"; + private static final String FILE_IO = "file-io"; + private static final String SCHEMA = "schema"; + private static final String SPECS = "partition-specs"; + private static final String MANIFEST_LIST_LOCATION = "manifest-list-Location"; + private static final String RESIDUAL = "residual-filter"; + private static final String REFERENCE_SNAPSHOT_ID = "reference-snapshot-id"; + + private AllManifestsTableTaskParser() {} + + static void toJson(AllManifestsTable.ManifestListReadTask task, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(task != null, "Invalid manifest task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(DATA_TABLE_SCHEMA); + SchemaParser.toJson(task.dataTableSchema(), generator); + + generator.writeFieldName(FILE_IO); + FileIOParser.toJson(task.io(), generator); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(task.schema(), generator); + + generator.writeArrayFieldStart(SPECS); + for (PartitionSpec spec : task.specsById().values()) { + PartitionSpecParser.toJson(spec, generator); + } + + generator.writeEndArray(); + + generator.writeStringField(MANIFEST_LIST_LOCATION, task.manifestListLocation()); + + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(task.residual(), generator); + + generator.writeNumberField(REFERENCE_SNAPSHOT_ID, task.referenceSnapshotId()); + } + + static AllManifestsTable.ManifestListReadTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for manifest task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for manifest task: non-object (%s)", jsonNode); + + Schema dataTableSchema = SchemaParser.fromJson(JsonUtil.get(DATA_TABLE_SCHEMA, jsonNode)); + FileIO fileIO = FileIOParser.fromJson(JsonUtil.get(FILE_IO, jsonNode), null); + Schema schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + + JsonNode specsArray = JsonUtil.get(SPECS, jsonNode); + Preconditions.checkArgument( + specsArray.isArray(), "Invalid JSON node for partition specs: non-array (%s)", specsArray); + + ImmutableList.Builder<PartitionSpec> specsBuilder = ImmutableList.builder(); + for (JsonNode specNode : specsArray) { + PartitionSpec spec = PartitionSpecParser.fromJson(dataTableSchema, specNode); + specsBuilder.add(spec); + } + + Map<Integer, PartitionSpec> specsById = PartitionUtil.indexSpecs(specsBuilder.build()); + String manifestListLocation = JsonUtil.getString(MANIFEST_LIST_LOCATION, jsonNode); + Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); + long referenceSnapshotId = JsonUtil.getLong(REFERENCE_SNAPSHOT_ID, jsonNode); + + return new AllManifestsTable.ManifestListReadTask( + dataTableSchema, + fileIO, + schema, + specsById, + manifestListLocation, + residualFilter, + referenceSnapshotId); + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index 93847d144ac9..5f3be15cefb9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -33,7 +33,6 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -92,15 +91,9 @@ static CloseableIterable<FileScanTask> planFiles( evalCache.get(manifest.partitionSpecId()).eval(manifest) && manifestContentEvaluator.eval(manifest)); - String schemaString = SchemaParser.toJson(projectedSchema); - String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter); - return CloseableIterable.transform( filteredManifests, - manifest -> - new ManifestReadTask( - table, manifest, projectedSchema, schemaString, specString, residuals)); + manifest -> new ManifestReadTask(table, manifest, projectedSchema, filter)); } /** @@ -292,7 +285,7 @@ private <T> boolean fileContent(BoundReference<T> ref) { return ref.fieldId() == DataFile.CONTENT.fieldId(); } - private <T> 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 @@ -313,19 +306,29 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask { private final ManifestFile manifest; private final Map<Integer, PartitionSpec> specsById; + private ManifestReadTask( + Table table, ManifestFile manifest, Schema projection, Expression filter) { + this(table.schema(), table.io(), table.specs(), manifest, projection, filter); + } + ManifestReadTask( - Table table, + Schema dataTableSchema, + FileIO io, + Map<Integer, PartitionSpec> specsById, ManifestFile manifest, Schema projection, - String schemaString, - String specString, - ResidualEvaluator residuals) { - super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals); + Expression filter) { + super( + DataFiles.fromManifest(manifest), + null, + SchemaParser.toJson(projection), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + ResidualEvaluator.unpartitioned(filter)); this.projection = projection; - this.io = table.io(); + this.io = io; this.manifest = manifest; - this.specsById = Maps.newHashMap(table.specs()); - this.dataTableSchema = table.schema(); + this.specsById = Maps.newHashMap(specsById); + this.dataTableSchema = dataTableSchema; Type fileProjectionType = projection.findType("data_file"); this.fileProjection = @@ -334,7 +337,13 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask { : new Schema(); } - @VisibleForTesting + @Override + public long estimatedRowsCount() { + return (long) manifest.addedFilesCount() + + (long) manifest.deletedFilesCount() + + (long) manifest.existingFilesCount(); + } + ManifestFile manifest() { return manifest; } @@ -426,5 +435,21 @@ private MetricsUtil.ReadableMetricsStruct readableMetrics( public Iterable<FileScanTask> split(long splitSize) { return ImmutableList.of(this); // don't split } + + FileIO io() { + return io; + } + + Map<Integer, PartitionSpec> specsById() { + return specsById; + } + + Schema dataTableSchema() { + return dataTableSchema; + } + + Schema projection() { + return projection; + } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java index 011abffd8a2a..b71744f739c7 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java @@ -29,7 +29,6 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -80,16 +79,11 @@ private static CloseableIterable<FileScanTask> planFiles( CloseableIterable.filter( manifests, manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); - String schemaString = SchemaParser.toJson(projectedSchema); - String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : rowFilter; - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter); return CloseableIterable.transform( filteredManifests, - manifest -> - new ManifestReadTask( - table, manifest, projectedSchema, schemaString, specString, residuals)); + manifest -> new ManifestReadTask(table, manifest, projectedSchema, filter)); } abstract static class BaseFilesTableScan extends BaseMetadataTableScan { @@ -140,18 +134,28 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask { private final Schema dataTableSchema; private final Schema projection; + private ManifestReadTask( + Table table, ManifestFile manifest, Schema projection, Expression filter) { + this(table.schema(), table.io(), table.specs(), manifest, projection, filter); + } + ManifestReadTask( - Table table, + Schema dataTableSchema, + FileIO io, + Map<Integer, PartitionSpec> specsById, ManifestFile manifest, Schema projection, - String schemaString, - String specString, - ResidualEvaluator residuals) { - super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals); - this.io = table.io(); - this.specsById = Maps.newHashMap(table.specs()); + Expression filter) { + super( + DataFiles.fromManifest(manifest), + null, + SchemaParser.toJson(projection), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + ResidualEvaluator.unpartitioned(filter)); + this.io = io; + this.specsById = Maps.newHashMap(specsById); this.manifest = manifest; - this.dataTableSchema = table.schema(); + this.dataTableSchema = dataTableSchema; this.projection = projection; } @@ -169,6 +173,13 @@ public CloseableIterable<StructLike> rows() { } } + @Override + public long estimatedRowsCount() { + return (long) manifest.addedFilesCount() + + (long) manifest.deletedFilesCount() + + (long) manifest.existingFilesCount(); + } + private CloseableIterable<? extends ContentFile<?>> files(Schema fileProjection) { switch (manifest.content()) { case DATA: @@ -233,9 +244,24 @@ public Iterable<FileScanTask> split(long splitSize) { return ImmutableList.of(this); // don't split } - @VisibleForTesting + FileIO io() { + return io; + } + + Map<Integer, PartitionSpec> specsById() { + return specsById; + } + ManifestFile manifest() { return manifest; } + + Schema dataTableSchema() { + return dataTableSchema; + } + + Schema projection() { + return projection; + } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 50266b2edcbf..5c82bc877a15 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -31,8 +31,10 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsBulkOperations; import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.Tasks; @@ -282,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 @@ -387,14 +378,21 @@ private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metada // the log, thus we don't include metadata.previousFiles() for deletion - everything else can // be removed removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); - Tasks.foreach(removedPreviousMetadataFiles) - .noRetry() - .suppressFailureWhenFinished() - .onFailure( - (previousMetadataFile, exc) -> - LOG.warn( - "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) - .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); + if (io() instanceof SupportsBulkOperations) { + ((SupportsBulkOperations) io()) + .deleteFiles( + Iterables.transform( + removedPreviousMetadataFiles, TableMetadata.MetadataLogEntry::file)); + } else { + Tasks.foreach(removedPreviousMetadataFiles) + .noRetry() + .suppressFailureWhenFinished() + .onFailure( + (previousMetadataFile, exc) -> + LOG.warn( + "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) + .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); + } } } } 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<String, String> 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<Integer, Transform<?, ?>> 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/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index c26716481836..794044373471 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -71,6 +71,8 @@ class DeleteFileIndex { private final PartitionMap<EqualityDeletes> eqDeletesByPartition; private final PartitionMap<PositionDeletes> posDeletesByPartition; private final CharSequenceMap<PositionDeletes> posDeletesByPath; + private final boolean hasEqDeletes; + private final boolean hasPosDeletes; private final boolean isEmpty; private DeleteFileIndex( @@ -82,15 +84,23 @@ private DeleteFileIndex( this.eqDeletesByPartition = eqDeletesByPartition; this.posDeletesByPartition = posDeletesByPartition; this.posDeletesByPath = posDeletesByPath; - boolean noEqDeletes = globalDeletes == null && eqDeletesByPartition == null; - boolean noPosDeletes = posDeletesByPartition == null && posDeletesByPath == null; - this.isEmpty = noEqDeletes && noPosDeletes; + this.hasEqDeletes = globalDeletes != null || eqDeletesByPartition != null; + this.hasPosDeletes = posDeletesByPartition != null || posDeletesByPath != null; + this.isEmpty = !hasEqDeletes && !hasPosDeletes; } public boolean isEmpty() { return isEmpty; } + public boolean hasEqualityDeletes() { + return hasEqDeletes; + } + + public boolean hasPositionDeletes() { + return hasPosDeletes; + } + public Iterable<DeleteFile> referencedDeleteFiles() { Iterable<DeleteFile> deleteFiles = Collections.emptyList(); diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1439289130d7..4976a8081c44 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -198,6 +198,16 @@ protected void cleanUncommitted(Set<ManifestFile> 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<ManifestFile> 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/FilesTableTaskParser.java b/core/src/main/java/org/apache/iceberg/FilesTableTaskParser.java new file mode 100644 index 000000000000..365deacebf10 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/FilesTableTaskParser.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; + +class FilesTableTaskParser { + private static final String FILE_IO = "file-io"; + private static final String SPECS = "partition-specs"; + private static final String SCHEMA = "schema"; + private static final String PROJECTION = "projection"; + private static final String RESIDUAL = "residual-filter"; + private static final String MANIFEST = "manifest-file"; + + private FilesTableTaskParser() {} + + static void toJson(BaseFilesTable.ManifestReadTask task, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(task != null, "Invalid files task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(task.dataTableSchema(), generator); + + generator.writeFieldName(PROJECTION); + SchemaParser.toJson(task.projection(), generator); + + generator.writeFieldName(FILE_IO); + FileIOParser.toJson(task.io(), generator); + + generator.writeArrayFieldStart(SPECS); + for (PartitionSpec spec : task.specsById().values()) { + PartitionSpecParser.toJson(spec, generator); + } + + generator.writeEndArray(); + + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(task.residual(), generator); + + generator.writeFieldName(MANIFEST); + ManifestFileParser.toJson(task.manifest(), generator); + } + + static BaseFilesTable.ManifestReadTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for files task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for files task: non-object (%s)", jsonNode); + + Schema dataTableSchema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + Schema projection = SchemaParser.fromJson(JsonUtil.get(PROJECTION, jsonNode)); + + FileIO fileIO = FileIOParser.fromJson(JsonUtil.get(FILE_IO, jsonNode), null); + + JsonNode specsArray = JsonUtil.get(SPECS, jsonNode); + Preconditions.checkArgument( + specsArray.isArray(), "Invalid JSON node for partition specs: non-array (%s)", specsArray); + + ImmutableList.Builder<PartitionSpec> specsBuilder = ImmutableList.builder(); + for (JsonNode specNode : specsArray) { + PartitionSpec spec = PartitionSpecParser.fromJson(dataTableSchema, specNode); + specsBuilder.add(spec); + } + + Map<Integer, PartitionSpec> specsById = PartitionUtil.indexSpecs(specsBuilder.build()); + Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); + ManifestFile manifestFile = ManifestFileParser.fromJson(JsonUtil.get(MANIFEST, jsonNode)); + + return new BaseFilesTable.ManifestReadTask( + dataTableSchema, fileIO, specsById, manifestFile, projection, residualFilter); + } +} diff --git a/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java b/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java index 46bedfa01753..d3ac399556c3 100644 --- a/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java +++ b/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -37,6 +38,13 @@ public static BlobMetadata from(org.apache.iceberg.puffin.BlobMetadata puffinMet puffinMetadata.properties()); } + public static List<BlobMetadata> from( + Collection<org.apache.iceberg.puffin.BlobMetadata> puffinMetadataList) { + return puffinMetadataList.stream() + .map(GenericBlobMetadata::from) + .collect(ImmutableList.toImmutableList()); + } + private final String type; private final long sourceSnapshotId; private final long sourceSnapshotSequenceNumber; diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index ed94da5e1791..d081e0bdd568 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -105,6 +105,42 @@ public GenericManifestFile(Schema avroSchema) { this.keyMetadata = null; } + /** Adjust the arg order to avoid conflict with the public constructor below */ + GenericManifestFile( + String path, + long length, + int specId, + ManifestContent content, + long sequenceNumber, + long minSequenceNumber, + Long snapshotId, + List<PartitionFieldSummary> partitions, + ByteBuffer keyMetadata, + Integer addedFilesCount, + Long addedRowsCount, + Integer existingFilesCount, + Long existingRowsCount, + Integer deletedFilesCount, + Long deletedRowsCount) { + this.avroSchema = AVRO_SCHEMA; + this.manifestPath = path; + this.length = length; + this.specId = specId; + this.content = content; + this.sequenceNumber = sequenceNumber; + this.minSequenceNumber = minSequenceNumber; + this.snapshotId = snapshotId; + this.addedFilesCount = addedFilesCount; + this.addedRowsCount = addedRowsCount; + this.existingFilesCount = existingFilesCount; + this.existingRowsCount = existingRowsCount; + this.deletedFilesCount = deletedFilesCount; + this.deletedRowsCount = deletedRowsCount; + this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); + this.fromProjectionPos = null; + this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); + } + public GenericManifestFile( String path, long length, diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java b/core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java new file mode 100644 index 000000000000..37f84d09ae91 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; + +class ManifestEntriesTableTaskParser { + private static final String FILE_IO = "file-io"; + private static final String SPECS = "partition-specs"; + private static final String SCHEMA = "schema"; + private static final String PROJECTION = "projection"; + private static final String RESIDUAL = "residual-filter"; + private static final String MANIFEST = "manifest-file"; + + private ManifestEntriesTableTaskParser() {} + + static void toJson(BaseEntriesTable.ManifestReadTask task, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(task != null, "Invalid manifest task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(task.dataTableSchema(), generator); + + generator.writeFieldName(FILE_IO); + FileIOParser.toJson(task.io(), generator); + + generator.writeArrayFieldStart(SPECS); + for (PartitionSpec spec : task.specsById().values()) { + PartitionSpecParser.toJson(spec, generator); + } + + generator.writeEndArray(); + + generator.writeFieldName(MANIFEST); + ManifestFileParser.toJson(task.manifest(), generator); + + generator.writeFieldName(PROJECTION); + SchemaParser.toJson(task.projection(), generator); + + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(task.residual(), generator); + } + + static BaseEntriesTable.ManifestReadTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for manifest task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for manifest task: non-object (%s)", jsonNode); + + Schema dataTableSchema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + FileIO fileIO = FileIOParser.fromJson(JsonUtil.get(FILE_IO, jsonNode), null); + + JsonNode specsArray = JsonUtil.get(SPECS, jsonNode); + Preconditions.checkArgument( + specsArray.isArray(), "Invalid JSON node for partition specs: non-array (%s)", specsArray); + ImmutableList.Builder<PartitionSpec> specsBuilder = ImmutableList.builder(); + for (JsonNode specNode : specsArray) { + PartitionSpec spec = PartitionSpecParser.fromJson(dataTableSchema, specNode); + specsBuilder.add(spec); + } + + Map<Integer, PartitionSpec> specsById = PartitionUtil.indexSpecs(specsBuilder.build()); + + ManifestFile manifestFile = ManifestFileParser.fromJson(JsonUtil.get(MANIFEST, jsonNode)); + Schema projection = SchemaParser.fromJson(JsonUtil.get(PROJECTION, jsonNode)); + Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); + + return new BaseEntriesTable.ManifestReadTask( + dataTableSchema, fileIO, specsById, manifestFile, projection, residualFilter); + } +} diff --git a/core/src/main/java/org/apache/iceberg/ManifestFileParser.java b/core/src/main/java/org/apache/iceberg/ManifestFileParser.java new file mode 100644 index 000000000000..80f1a0fcf7fc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ManifestFileParser.java @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; + +class ManifestFileParser { + private static final String PATH = "path"; + private static final String LENGTH = "length"; + private static final String SPEC_ID = "partition-spec-id"; + private static final String CONTENT = "content"; + private static final String SEQUENCE_NUMBER = "sequence-number"; + private static final String MIN_SEQUENCE_NUMBER = "min-sequence-number"; + private static final String ADDED_SNAPSHOT_ID = "added-snapshot-id"; + private static final String ADDED_FILES_COUNT = "added-files-count"; + private static final String EXISTING_FILES_COUNT = "existing-files-count"; + private static final String DELETED_FILES_COUNT = "deleted-files-count"; + private static final String ADDED_ROWS_COUNT = "added-rows-count"; + private static final String EXISTING_ROWS_COUNT = "existing-rows-count"; + private static final String DELETED_ROWS_COUNT = "deleted-rows-count"; + private static final String PARTITION_FIELD_SUMMARY = "partition-field-summary"; + private static final String KEY_METADATA = "key-metadata"; + + private ManifestFileParser() {} + + static void toJson(ManifestFile manifestFile, JsonGenerator generator) throws IOException { + Preconditions.checkArgument(manifestFile != null, "Invalid manifest file: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeStartObject(); + + generator.writeStringField(PATH, manifestFile.path()); + generator.writeNumberField(LENGTH, manifestFile.length()); + generator.writeNumberField(SPEC_ID, manifestFile.partitionSpecId()); + + if (manifestFile.content() != null) { + generator.writeNumberField(CONTENT, manifestFile.content().id()); + } + + generator.writeNumberField(SEQUENCE_NUMBER, manifestFile.sequenceNumber()); + generator.writeNumberField(MIN_SEQUENCE_NUMBER, manifestFile.minSequenceNumber()); + + if (manifestFile.snapshotId() != null) { + generator.writeNumberField(ADDED_SNAPSHOT_ID, manifestFile.snapshotId()); + } + + if (manifestFile.addedFilesCount() != null) { + generator.writeNumberField(ADDED_FILES_COUNT, manifestFile.addedFilesCount()); + } + + if (manifestFile.existingFilesCount() != null) { + generator.writeNumberField(EXISTING_FILES_COUNT, manifestFile.existingFilesCount()); + } + + if (manifestFile.deletedFilesCount() != null) { + generator.writeNumberField(DELETED_FILES_COUNT, manifestFile.deletedFilesCount()); + } + + if (manifestFile.addedRowsCount() != null) { + generator.writeNumberField(ADDED_ROWS_COUNT, manifestFile.addedRowsCount()); + } + + if (manifestFile.existingRowsCount() != null) { + generator.writeNumberField(EXISTING_ROWS_COUNT, manifestFile.existingRowsCount()); + } + + if (manifestFile.deletedRowsCount() != null) { + generator.writeNumberField(DELETED_ROWS_COUNT, manifestFile.deletedRowsCount()); + } + + if (manifestFile.partitions() != null) { + generator.writeArrayFieldStart(PARTITION_FIELD_SUMMARY); + for (ManifestFile.PartitionFieldSummary summary : manifestFile.partitions()) { + PartitionFieldSummaryParser.toJson(summary, generator); + } + + generator.writeEndArray(); + } + + if (manifestFile.keyMetadata() != null) { + generator.writeFieldName(KEY_METADATA); + SingleValueParser.toJson(DataFile.KEY_METADATA.type(), manifestFile.keyMetadata(), generator); + } + + generator.writeEndObject(); + } + + static ManifestFile fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for manifest file: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for manifest file: non-object (%s)", jsonNode); + + String path = JsonUtil.getString(PATH, jsonNode); + long length = JsonUtil.getLong(LENGTH, jsonNode); + int specId = JsonUtil.getInt(SPEC_ID, jsonNode); + + ManifestContent manifestContent = null; + if (jsonNode.has(CONTENT)) { + manifestContent = ManifestContent.fromId(JsonUtil.getInt(CONTENT, jsonNode)); + } + + long sequenceNumber = JsonUtil.getLong(SEQUENCE_NUMBER, jsonNode); + long minSequenceNumber = JsonUtil.getLong(MIN_SEQUENCE_NUMBER, jsonNode); + + Long addedSnapshotId = null; + if (jsonNode.has(ADDED_SNAPSHOT_ID)) { + addedSnapshotId = JsonUtil.getLong(ADDED_SNAPSHOT_ID, jsonNode); + } + + Integer addedFilesCount = null; + if (jsonNode.has(ADDED_FILES_COUNT)) { + addedFilesCount = JsonUtil.getInt(ADDED_FILES_COUNT, jsonNode); + } + + Integer existingFilesCount = null; + if (jsonNode.has(EXISTING_FILES_COUNT)) { + existingFilesCount = JsonUtil.getInt(EXISTING_FILES_COUNT, jsonNode); + } + + Integer deletedFilesCount = null; + if (jsonNode.has(DELETED_FILES_COUNT)) { + deletedFilesCount = JsonUtil.getInt(DELETED_FILES_COUNT, jsonNode); + } + + Long addedRowsCount = null; + if (jsonNode.has(ADDED_ROWS_COUNT)) { + addedRowsCount = JsonUtil.getLong(ADDED_ROWS_COUNT, jsonNode); + } + + Long existingRowsCount = null; + if (jsonNode.has(EXISTING_ROWS_COUNT)) { + existingRowsCount = JsonUtil.getLong(EXISTING_ROWS_COUNT, jsonNode); + } + + Long deletedRowsCount = null; + if (jsonNode.has(DELETED_ROWS_COUNT)) { + deletedRowsCount = JsonUtil.getLong(DELETED_ROWS_COUNT, jsonNode); + } + + List<ManifestFile.PartitionFieldSummary> partitionFieldSummaries = null; + if (jsonNode.has(PARTITION_FIELD_SUMMARY)) { + JsonNode summaryArray = JsonUtil.get(PARTITION_FIELD_SUMMARY, jsonNode); + Preconditions.checkArgument( + summaryArray.isArray(), + "Invalid JSON node for partition field summaries: non-array (%s)", + summaryArray); + + ImmutableList.Builder<ManifestFile.PartitionFieldSummary> builder = ImmutableList.builder(); + for (JsonNode summaryNode : summaryArray) { + ManifestFile.PartitionFieldSummary summary = + PartitionFieldSummaryParser.fromJson(summaryNode); + builder.add(summary); + } + + partitionFieldSummaries = builder.build(); + } + + ByteBuffer keyMetadata = JsonUtil.getByteBufferOrNull(KEY_METADATA, jsonNode); + + return new GenericManifestFile( + path, + length, + specId, + manifestContent, + sequenceNumber, + minSequenceNumber, + addedSnapshotId, + partitionFieldSummaries, + keyMetadata, + addedFilesCount, + addedRowsCount, + existingFilesCount, + existingRowsCount, + deletedFilesCount, + deletedRowsCount); + } + + private static class PartitionFieldSummaryParser { + private static final String CONTAINS_NULL = "contains-null"; + private static final String CONTAINS_NAN = "contains-nan"; + private static final String LOWER_BOUND = "lower-bound"; + private static final String UPPER_BOUND = "upper-bound"; + + private PartitionFieldSummaryParser() {} + + static void toJson(ManifestFile.PartitionFieldSummary summary, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(summary != null, "Invalid partition field summary: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeStartObject(); + + generator.writeBooleanField(CONTAINS_NULL, summary.containsNull()); + + if (summary.containsNaN() != null) { + generator.writeBooleanField(CONTAINS_NAN, summary.containsNaN()); + } + + if (summary.lowerBound() != null) { + generator.writeFieldName(LOWER_BOUND); + SingleValueParser.toJson(Types.BinaryType.get(), summary.lowerBound(), generator); + } + + if (summary.upperBound() != null) { + generator.writeFieldName(UPPER_BOUND); + SingleValueParser.toJson(Types.BinaryType.get(), summary.upperBound(), generator); + } + + generator.writeEndObject(); + } + + static ManifestFile.PartitionFieldSummary fromJson(JsonNode jsonNode) { + Preconditions.checkArgument( + jsonNode != null, "Invalid JSON node for partition field summary: null"); + Preconditions.checkArgument( + jsonNode.isObject(), + "Invalid JSON node for partition field summary: non-object (%s)", + jsonNode); + + boolean containsNull = JsonUtil.getBool(CONTAINS_NULL, jsonNode); + Boolean containsNaN = null; + if (jsonNode.has(CONTAINS_NAN)) { + containsNaN = JsonUtil.getBool(CONTAINS_NAN, jsonNode); + } + + ByteBuffer lowerBound = null; + if (jsonNode.has(LOWER_BOUND)) { + lowerBound = + (ByteBuffer) + SingleValueParser.fromJson(Types.BinaryType.get(), jsonNode.get(LOWER_BOUND)); + } + + ByteBuffer upperBound = null; + if (jsonNode.has(UPPER_BOUND)) { + upperBound = + (ByteBuffer) + SingleValueParser.fromJson(Types.BinaryType.get(), jsonNode.get(UPPER_BOUND)); + } + + if (containsNaN != null) { + return new GenericPartitionFieldSummary(containsNull, containsNaN, lowerBound, upperBound); + } else { + return new GenericPartitionFieldSummary(containsNull, lowerBound, upperBound); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 9009f19ec947..f630bb3eb743 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -81,7 +81,7 @@ static ContentCache contentCache(FileIO io) { } /** Drop manifest file cache object for a FileIO if exists. */ - public static synchronized void dropCache(FileIO fileIO) { + public static void dropCache(FileIO fileIO) { CONTENT_CACHES.invalidate(fileIO); CONTENT_CACHES.cleanUp(); } @@ -182,6 +182,8 @@ public static ManifestWriter<DataFile> 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<DeleteFile> 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/ManifestGroup.java b/core/src/main/java/org/apache/iceberg/ManifestGroup.java index 38ad2661a8a3..10cf6bd3bfbd 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestGroup.java +++ b/core/src/main/java/org/apache/iceberg/ManifestGroup.java @@ -184,7 +184,7 @@ public <T extends ScanTask> CloseableIterable<T> plan(CreateTasksFunction<T> cre DeleteFileIndex deleteFiles = deleteIndexBuilder.scanMetrics(scanMetrics).build(); boolean dropStats = ManifestReader.dropStats(columns); - if (!deleteFiles.isEmpty()) { + if (deleteFiles.hasEqualityDeletes()) { select(ManifestReader.withStatsColumns(columns)); } diff --git a/core/src/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<ManifestFile> newAppender(OutputFile file, Map<String, String> 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<DataFile> { + private final V3Metadata.IndexedManifestEntry<DataFile> entryWrapper; + + V3Writer(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry<DataFile> prepare(ManifestEntry<DataFile> entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender<ManifestEntry<DataFile>> 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<DeleteFile> { + private final V3Metadata.IndexedManifestEntry<DeleteFile> entryWrapper; + + V3DeleteWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry<DeleteFile> prepare(ManifestEntry<DeleteFile> entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender<ManifestEntry<DeleteFile>> 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<DataFile> { private final V2Metadata.IndexedManifestEntry<DataFile> entryWrapper; diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 1a4560416dc2..b4c0567ab73a 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.exceptions.RuntimeIOException; @@ -42,6 +43,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Predicate; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; @@ -79,7 +81,7 @@ abstract class MergingSnapshotProducer<ThisT> extends SnapshotProducer<ThisT> { private final ManifestFilterManager<DeleteFile> deleteFilterManager; // update data - private final List<DataFile> newDataFiles = Lists.newArrayList(); + private final Map<PartitionSpec, List<DataFile>> newDataFilesBySpec = Maps.newHashMap(); private final CharSequenceSet newDataFilePaths = CharSequenceSet.empty(); private final CharSequenceSet newDeleteFilePaths = CharSequenceSet.empty(); private Long newDataFilesDataSequenceNumber; @@ -89,10 +91,9 @@ abstract class MergingSnapshotProducer<ThisT> extends SnapshotProducer<ThisT> { private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); private final SnapshotSummary.Builder appendedManifestsSummary = SnapshotSummary.builder(); private Expression deleteExpression = Expressions.alwaysFalse(); - private PartitionSpec dataSpec; // cache new data manifests after writing - private List<ManifestFile> cachedNewDataManifests = null; + private final List<ManifestFile> cachedNewDataManifests = Lists.newLinkedList(); private boolean hasNewDataFiles = false; // cache new manifests for delete files @@ -105,7 +106,6 @@ abstract class MergingSnapshotProducer<ThisT> extends SnapshotProducer<ThisT> { super(ops); this.tableName = tableName; this.ops = ops; - this.dataSpec = null; long targetSizeBytes = ops.current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); @@ -141,10 +141,18 @@ protected boolean isCaseSensitive() { } protected PartitionSpec dataSpec() { + Set<PartitionSpec> specs = dataSpecs(); Preconditions.checkState( - dataSpec != null, "Cannot determine partition spec: no data files have been added"); - // the spec is set when the write is started - return dataSpec; + specs.size() == 1, + "Cannot return a single partition spec: data files with different partition specs have been added"); + return specs.iterator().next(); + } + + protected Set<PartitionSpec> dataSpecs() { + Set<PartitionSpec> specs = newDataFilesBySpec.keySet(); + Preconditions.checkState( + !specs.isEmpty(), "Cannot determine partition specs: no data files have been added"); + return ImmutableSet.copyOf(specs); } protected Expression rowFilter() { @@ -152,7 +160,12 @@ protected Expression rowFilter() { } protected List<DataFile> addedDataFiles() { - return ImmutableList.copyOf(newDataFiles); + return ImmutableList.copyOf( + newDataFilesBySpec.values().stream().flatMap(List::stream).collect(Collectors.toList())); + } + + protected Map<PartitionSpec, List<DataFile>> addedDataFilesBySpec() { + return ImmutableMap.copyOf(newDataFilesBySpec); } protected void failAnyDelete() { @@ -212,7 +225,7 @@ protected boolean deletesDeleteFiles() { } protected boolean addsDataFiles() { - return !newDataFiles.isEmpty(); + return !newDataFilesBySpec.isEmpty(); } protected boolean addsDeleteFiles() { @@ -223,9 +236,17 @@ protected boolean addsDeleteFiles() { protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); if (newDataFilePaths.add(file.path())) { - setDataSpec(file); - addedFilesSummary.addedFile(dataSpec(), file); + PartitionSpec fileSpec = ops.current().spec(file.specId()); + Preconditions.checkArgument( + fileSpec != null, + "Cannot find partition spec %s for data file: %s", + file.specId(), + file.path()); + + addedFilesSummary.addedFile(fileSpec, file); hasNewDataFiles = true; + List<DataFile> newDataFiles = + newDataFilesBySpec.computeIfAbsent(fileSpec, ignored -> Lists.newArrayList()); newDataFiles.add(file); } } @@ -255,17 +276,6 @@ private void add(DeleteFileHolder fileHolder) { } } - private void setDataSpec(DataFile file) { - PartitionSpec fileSpec = ops.current().spec(file.specId()); - Preconditions.checkNotNull( - fileSpec, "Cannot find partition spec for data file: %s", file.path()); - if (dataSpec == null) { - dataSpec = fileSpec; - } else if (dataSpec.specId() != file.specId()) { - throw new ValidationException("Invalid data file, expected spec id: %d", dataSpec.specId()); - } - } - /** Add all files in a manifest to the new snapshot. */ protected void add(ManifestFile manifest) { Preconditions.checkArgument( @@ -885,7 +895,7 @@ public Object updateEvent() { @SuppressWarnings("checkstyle:CyclomaticComplexity") private void cleanUncommittedAppends(Set<ManifestFile> committed) { - if (cachedNewDataManifests != null) { + if (!cachedNewDataManifests.isEmpty()) { boolean hasDeletes = false; for (ManifestFile manifest : cachedNewDataManifests) { if (!committed.contains(manifest)) { @@ -895,7 +905,7 @@ private void cleanUncommittedAppends(Set<ManifestFile> committed) { } if (hasDeletes) { - this.cachedNewDataManifests = null; + this.cachedNewDataManifests.clear(); } } @@ -941,7 +951,7 @@ protected void cleanUncommitted(Set<ManifestFile> committed) { private Iterable<ManifestFile> prepareNewDataManifests() { Iterable<ManifestFile> newManifests; - if (!newDataFiles.isEmpty()) { + if (!newDataFilesBySpec.isEmpty()) { List<ManifestFile> dataFileManifests = newDataFilesAsManifests(); newManifests = Iterables.concat(dataFileManifests, appendManifests, rewrittenAppendManifests); } else { @@ -954,29 +964,31 @@ private Iterable<ManifestFile> prepareNewDataManifests() { } private List<ManifestFile> newDataFilesAsManifests() { - if (hasNewDataFiles && cachedNewDataManifests != null) { + if (hasNewDataFiles && !cachedNewDataManifests.isEmpty()) { cachedNewDataManifests.forEach(file -> deleteFile(file.path())); - cachedNewDataManifests = null; + cachedNewDataManifests.clear(); } - if (cachedNewDataManifests == null) { - try { - RollingManifestWriter<DataFile> writer = newRollingManifestWriter(dataSpec()); - try { - if (newDataFilesDataSequenceNumber == null) { - newDataFiles.forEach(writer::add); - } else { - newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); - } - } finally { - writer.close(); - } - - this.cachedNewDataManifests = writer.toManifestFiles(); - this.hasNewDataFiles = false; - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close manifest writer"); - } + if (cachedNewDataManifests.isEmpty()) { + newDataFilesBySpec.forEach( + (dataSpec, newDataFiles) -> { + try { + RollingManifestWriter<DataFile> writer = newRollingManifestWriter(dataSpec); + try { + if (newDataFilesDataSequenceNumber == null) { + newDataFiles.forEach(writer::add); + } else { + newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); + } + } finally { + writer.close(); + } + this.cachedNewDataManifests.addAll(writer.toManifestFiles()); + this.hasNewDataFiles = false; + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to close manifest writer"); + } + }); } return cachedNewDataManifests; diff --git a/core/src/main/java/org/apache/iceberg/ScanTaskParser.java b/core/src/main/java/org/apache/iceberg/ScanTaskParser.java index 9447d0668a1f..67e44cea7d07 100644 --- a/core/src/main/java/org/apache/iceberg/ScanTaskParser.java +++ b/core/src/main/java/org/apache/iceberg/ScanTaskParser.java @@ -30,7 +30,10 @@ public class ScanTaskParser { private enum TaskType { FILE_SCAN_TASK("file-scan-task"), - DATA_TASK("data-task"); + DATA_TASK("data-task"), + FILES_TABLE_TASK("files-table-task"), + ALL_MANIFESTS_TABLE_TASK("all-manifests-table-task"), + MANIFEST_ENTRIES_TABLE_TASK("manifest-entries-task"); private final String value; @@ -45,6 +48,12 @@ public static TaskType fromTypeName(String value) { return FILE_SCAN_TASK; } else if (DATA_TASK.typeName().equalsIgnoreCase(value)) { return DATA_TASK; + } else if (FILES_TABLE_TASK.typeName().equalsIgnoreCase(value)) { + return FILES_TABLE_TASK; + } else if (ALL_MANIFESTS_TABLE_TASK.typeName().equalsIgnoreCase(value)) { + return ALL_MANIFESTS_TABLE_TASK; + } else if (MANIFEST_ENTRIES_TABLE_TASK.typeName().equalsIgnoreCase(value)) { + return MANIFEST_ENTRIES_TABLE_TASK; } else { throw new IllegalArgumentException("Unknown task type: " + value); } @@ -74,6 +83,17 @@ private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) if (fileScanTask instanceof StaticDataTask) { generator.writeStringField(TASK_TYPE, TaskType.DATA_TASK.typeName()); DataTaskParser.toJson((StaticDataTask) fileScanTask, generator); + } else if (fileScanTask instanceof BaseFilesTable.ManifestReadTask) { + generator.writeStringField(TASK_TYPE, TaskType.FILES_TABLE_TASK.typeName()); + FilesTableTaskParser.toJson((BaseFilesTable.ManifestReadTask) fileScanTask, generator); + } else if (fileScanTask instanceof AllManifestsTable.ManifestListReadTask) { + generator.writeStringField(TASK_TYPE, TaskType.ALL_MANIFESTS_TABLE_TASK.typeName()); + AllManifestsTableTaskParser.toJson( + (AllManifestsTable.ManifestListReadTask) fileScanTask, generator); + } else if (fileScanTask instanceof BaseEntriesTable.ManifestReadTask) { + generator.writeStringField(TASK_TYPE, TaskType.MANIFEST_ENTRIES_TABLE_TASK.typeName()); + ManifestEntriesTableTaskParser.toJson( + (BaseEntriesTable.ManifestReadTask) fileScanTask, generator); } else if (fileScanTask instanceof BaseFileScanTask || fileScanTask instanceof BaseFileScanTask.SplitScanTask) { generator.writeStringField(TASK_TYPE, TaskType.FILE_SCAN_TASK.typeName()); @@ -98,6 +118,12 @@ private static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { return FileScanTaskParser.fromJson(jsonNode, caseSensitive); case DATA_TASK: return DataTaskParser.fromJson(jsonNode); + case FILES_TABLE_TASK: + return FilesTableTaskParser.fromJson(jsonNode); + case ALL_MANIFESTS_TABLE_TASK: + return AllManifestsTableTaskParser.fromJson(jsonNode); + case MANIFEST_ENTRIES_TABLE_TASK: + return ManifestEntriesTableTaskParser.fromJson(jsonNode); default: throw new UnsupportedOperationException("Unsupported task type: " + taskType.typeName()); } diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 9e0055a10376..082e50b840dc 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -278,6 +278,16 @@ public TableScan newScan() { return lazyTable().newScan(); } + @Override + public IncrementalAppendScan newIncrementalAppendScan() { + return lazyTable().newIncrementalAppendScan(); + } + + @Override + public IncrementalChangelogScan newIncrementalChangelogScan() { + return lazyTable().newIncrementalChangelogScan(); + } + @Override public BatchScan newBatchScan() { return lazyTable().newBatchScan(); 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<Snapshot> 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<DataFile> reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById())) { 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 e8dcfc85fbf0..c0bb16bc109d 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -43,6 +43,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SerializableSupplier; @@ -51,7 +52,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; @@ -331,7 +332,7 @@ public String toString() { this.snapshotsById = indexAndValidateSnapshots(snapshots, lastSequenceNumber); this.schemasById = indexSchemas(); - this.specsById = indexSpecs(specs); + this.specsById = PartitionUtil.indexSpecs(specs); this.sortOrdersById = indexSortOrders(sortOrders); this.refs = validateRefs(currentSnapshotId, refs, snapshotsById); this.statisticsFiles = ImmutableList.copyOf(statisticsFiles); @@ -564,6 +565,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(); } @@ -806,14 +811,6 @@ private Map<Integer, Schema> indexSchemas() { return builder.build(); } - private static Map<Integer, PartitionSpec> indexSpecs(List<PartitionSpec> specs) { - ImmutableMap.Builder<Integer, PartitionSpec> builder = ImmutableMap.builder(); - for (PartitionSpec spec : specs) { - builder.put(spec.specId(), spec); - } - return builder.build(); - } - private static Map<Integer, SortOrder> indexSortOrders(List<SortOrder> sortOrders) { ImmutableMap.Builder<Integer, SortOrder> builder = ImmutableMap.builder(); for (SortOrder sortOrder : sortOrders) { @@ -910,7 +907,7 @@ private Builder() { this(DEFAULT_TABLE_FORMAT_VERSION); } - public Builder(int formatVersion) { + private Builder(int formatVersion) { this.base = null; this.formatVersion = formatVersion; this.lastSequenceNumber = INITIAL_SEQUENCE_NUMBER; @@ -1050,7 +1047,7 @@ public Builder setCurrentSchema(int schemaId) { this.specs = Lists.newArrayList(Iterables.transform(specs, spec -> updateSpecSchema(schema, spec))); specsById.clear(); - specsById.putAll(indexSpecs(specs)); + specsById.putAll(PartitionUtil.indexSpecs(specs)); this.sortOrders = Lists.newArrayList( @@ -1497,6 +1494,8 @@ private int addSchemaInternal(Schema schema, int newLastColumnId) { newLastColumnId, lastColumnId); + Schema.checkCompatibility(schema, formatVersion); + int newSchemaId = reuseOrCreateNewSchemaId(schema); boolean schemaFound = schemasById.containsKey(newSchemaId); if (schemaFound && newLastColumnId == lastColumnId) { diff --git a/core/src/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. + * + * <p>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<PartitionFieldSummary> 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<F extends ContentFile<F>> + implements ManifestEntry<F>, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final Long commitSnapshotId; + private final IndexedDataFile<?> fileWrapper; + private ManifestEntry<F> 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<F> wrap(ManifestEntry<F> 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<F> copy() { + return wrapped.copy(); + } + + @Override + public ManifestEntry<F> copyWithoutStats() { + return wrapped.copyWithoutStats(); + } + } + + /** Wrapper used to write DataFile or DeleteFile to v3 metadata. */ + static class IndexedDataFile<F> implements ContentFile<F>, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final IndexedStructLike partitionWrapper; + private ContentFile<F> 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<F> wrap(ContentFile<?> file) { + this.wrapped = (ContentFile<F>) 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<Integer, Long> columnSizes() { + return wrapped.columnSizes(); + } + + @Override + public Map<Integer, Long> valueCounts() { + return wrapped.valueCounts(); + } + + @Override + public Map<Integer, Long> nullValueCounts() { + return wrapped.nullValueCounts(); + } + + @Override + public Map<Integer, Long> nanValueCounts() { + return wrapped.nanValueCounts(); + } + + @Override + public Map<Integer, ByteBuffer> lowerBounds() { + return wrapped.lowerBounds(); + } + + @Override + public Map<Integer, ByteBuffer> upperBounds() { + return wrapped.upperBounds(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public List<Long> splitOffsets() { + return wrapped.splitOffsets(); + } + + @Override + public List<Integer> 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<Integer> 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/BaseComputeTableStats.java b/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java new file mode 100644 index 000000000000..71941af1d7f0 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import javax.annotation.Nullable; +import org.apache.iceberg.StatisticsFile; +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableComputeTableStats", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseComputeTableStats extends ComputeTableStats { + + @Value.Immutable + interface Result extends ComputeTableStats.Result { + @Override + @Nullable + StatisticsFile statisticsFile(); + } +} diff --git a/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/avro/AvroIO.java b/core/src/main/java/org/apache/iceberg/avro/AvroIO.java index cf575fb0e8bf..ef263853190a 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroIO.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroIO.java @@ -43,26 +43,26 @@ class AvroIO { private AvroIO() {} - private static final Class<?> fsDataInputStreamClass = + private static final Class<?> FS_DATA_INPUT_STREAM_CLASS = DynClasses.builder().impl("org.apache.hadoop.fs.FSDataInputStream").orNull().build(); - private static final boolean relocated = + private static final boolean RELOCATED = "org.apache.avro.file.SeekableInput".equals(SeekableInput.class.getName()); - private static final DynConstructors.Ctor<SeekableInput> avroFsInputCtor = - !relocated && fsDataInputStreamClass != null + private static final DynConstructors.Ctor<SeekableInput> AVRO_FS_INPUT_CTOR = + !RELOCATED && FS_DATA_INPUT_STREAM_CLASS != null ? DynConstructors.builder(SeekableInput.class) - .impl("org.apache.hadoop.fs.AvroFSInput", fsDataInputStreamClass, Long.TYPE) + .impl("org.apache.hadoop.fs.AvroFSInput", FS_DATA_INPUT_STREAM_CLASS, Long.TYPE) .build() : null; static SeekableInput stream(SeekableInputStream stream, long length) { if (stream instanceof DelegatingInputStream) { InputStream wrapped = ((DelegatingInputStream) stream).getDelegate(); - if (avroFsInputCtor != null - && fsDataInputStreamClass != null - && fsDataInputStreamClass.isInstance(wrapped)) { - return avroFsInputCtor.newInstance(wrapped, length); + if (AVRO_FS_INPUT_CTOR != null + && FS_DATA_INPUT_STREAM_CLASS != null + && FS_DATA_INPUT_STREAM_CLASS.isInstance(wrapped)) { + return AVRO_FS_INPUT_CTOR.newInstance(wrapped, length); } } return new AvroInputStreamAdapter(stream, length); diff --git a/core/src/main/java/org/apache/iceberg/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<T extends StructLike> extends Filter<T> { @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/encryption/StandardKeyMetadata.java b/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java index 08466f75fe21..98f87c65d95f 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java @@ -40,8 +40,8 @@ class StandardKeyMetadata implements NativeEncryptionKeyMetadata, IndexedRecord private static final org.apache.avro.Schema AVRO_SCHEMA_V1 = AvroSchemaUtil.convert(SCHEMA_V1, StandardKeyMetadata.class.getCanonicalName()); - private static final Map<Byte, Schema> schemaVersions = ImmutableMap.of(V1, SCHEMA_V1); - private static final Map<Byte, org.apache.avro.Schema> avroSchemaVersions = + private static final Map<Byte, Schema> SCHEMA_VERSIONS = ImmutableMap.of(V1, SCHEMA_V1); + private static final Map<Byte, org.apache.avro.Schema> AVRO_SCHEMA_VERSIONS = ImmutableMap.of(V1, AVRO_SCHEMA_V1); private static final KeyMetadataEncoder KEY_METADATA_ENCODER = new KeyMetadataEncoder(V1); @@ -66,11 +66,11 @@ private StandardKeyMetadata(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { } static Map<Byte, Schema> supportedSchemaVersions() { - return schemaVersions; + return SCHEMA_VERSIONS; } static Map<Byte, org.apache.avro.Schema> supportedAvroSchemaVersions() { - return avroSchemaVersions; + return AVRO_SCHEMA_VERSIONS; } @Override diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java index c817b1d90afb..6a48b14c227d 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java @@ -122,7 +122,7 @@ public int read(byte[] b, int off, int len) throws IOException { return stream.read(b, off, len); } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); @@ -195,7 +195,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/core/src/main/java/org/apache/iceberg/io/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<String, FileContent> 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<ByteBuffer> buffers; diff --git a/core/src/main/java/org/apache/iceberg/io/FileIOParser.java b/core/src/main/java/org/apache/iceberg/io/FileIOParser.java index 90ace239c2b7..2743b7b616cd 100644 --- a/core/src/main/java/org/apache/iceberg/io/FileIOParser.java +++ b/core/src/main/java/org/apache/iceberg/io/FileIOParser.java @@ -40,7 +40,7 @@ public static String toJson(FileIO io, boolean pretty) { return JsonUtil.generate(gen -> toJson(io, gen), pretty); } - private static void toJson(FileIO io, JsonGenerator generator) throws IOException { + public static void toJson(FileIO io, JsonGenerator generator) throws IOException { String impl = io.getClass().getName(); Map<String, String> properties; try { @@ -72,7 +72,7 @@ public static FileIO fromJson(String json, Object conf) { return JsonUtil.parse(json, node -> fromJson(node, conf)); } - private static FileIO fromJson(JsonNode json, Object conf) { + public static FileIO fromJson(JsonNode json, Object conf) { Preconditions.checkArgument(json.isObject(), "Cannot parse FileIO from non-object: %s", json); String impl = JsonUtil.getString(FILE_IO_IMPL, json); Map<String, String> properties = JsonUtil.getStringMap(PROPERTIES, json); diff --git a/core/src/main/java/org/apache/iceberg/io/FileIOTracker.java b/core/src/main/java/org/apache/iceberg/io/FileIOTracker.java new file mode 100644 index 000000000000..9d8630e79b14 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/FileIOTracker.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.io; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalListener; +import java.io.Closeable; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Keeps track of the {@link FileIO} instance of the given {@link TableOperations} instance and + * closes the {@link FileIO} when {@link FileIOTracker#close()} gets called + */ +public class FileIOTracker implements Closeable { + private final Cache<TableOperations, FileIO> tracker; + + public FileIOTracker() { + this.tracker = + Caffeine.newBuilder() + .weakKeys() + .removalListener( + (RemovalListener<TableOperations, FileIO>) + (ops, fileIO, cause) -> { + if (null != fileIO) { + fileIO.close(); + } + }) + .build(); + } + + public void track(TableOperations ops) { + Preconditions.checkArgument(null != ops, "Invalid table ops: null"); + tracker.put(ops, ops.io()); + } + + @VisibleForTesting + Cache<TableOperations, FileIO> tracker() { + return tracker; + } + + @Override + public void close() { + tracker.invalidateAll(); + tracker.cleanUp(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java index e5c7c5163129..a858045aab8b 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -21,8 +21,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; @@ -141,7 +143,7 @@ public void close() { @Override public void serializeConfWith( Function<Configuration, SerializableSupplier<Configuration>> confSerializer) { - this.hadoopConf = confSerializer.apply(hadoopConf.get()); + this.hadoopConf = confSerializer.apply(getConf()); } @Override @@ -151,7 +153,7 @@ public void setConf(Configuration conf) { @Override public Configuration getConf() { - return hadoopConf.get(); + return Optional.ofNullable(hadoopConf).map(Supplier::get).orElse(null); } @VisibleForTesting @@ -163,7 +165,7 @@ DelegateFileIO io(String location) { synchronized (io) { if (((HadoopConfigurable) io).getConf() == null) { // re-apply the config in case it's null after Kryo serialization - ((HadoopConfigurable) io).setConf(hadoopConf.get()); + ((HadoopConfigurable) io).setConf(getConf()); } } } @@ -174,7 +176,7 @@ DelegateFileIO io(String location) { return ioInstances.computeIfAbsent( impl, key -> { - Configuration conf = hadoopConf.get(); + Configuration conf = getConf(); FileIO fileIO; try { @@ -240,7 +242,7 @@ private static String scheme(String location) { return null; } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 4e10ee96d125..5e985faafbed 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -468,6 +468,16 @@ public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespac .toArray(String[]::new))) // remove duplicates .distinct() + // exclude fuzzy matches when `namespace` contains `%` or `_` + .filter( + n -> { + for (int i = 0; i < namespace.levels().length; i++) { + if (!n.levels()[i].equals(namespace.levels()[i])) { + return false; + } + } + return true; + }) .collect(Collectors.toList()); return namespaces; diff --git a/core/src/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 56e07202f873..53ce45bb0a3f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -47,7 +47,6 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; import org.apache.iceberg.Transactions; import org.apache.iceberg.catalog.BaseViewSessionCatalog; @@ -63,6 +62,7 @@ import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOTracker; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.MetricsReporters; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -136,7 +136,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private final BiFunction<SessionContext, Map<String, String>, FileIO> ioBuilder; private Cache<String, AuthSession> sessions = null; private Cache<String, AuthSession> tableSessions = null; - private Cache<TableOperations, FileIO> fileIOCloser; + private FileIOTracker fileIOTracker = null; private AuthSession catalogAuth = null; private boolean keepTokenRefreshed = true; private RESTClient client = null; @@ -184,19 +184,34 @@ public void initialize(String name, Map<String, String> unresolved) { long startTimeMillis = System.currentTimeMillis(); // keep track of the init start time for token refresh String initToken = props.get(OAuth2Properties.TOKEN); + boolean hasInitToken = initToken != null; // fetch auth and config to complete initialization ConfigResponse config; OAuthTokenResponse authResponse; String credential = props.get(OAuth2Properties.CREDENTIAL); + boolean hasCredential = credential != null && !credential.isEmpty(); String scope = props.getOrDefault(OAuth2Properties.SCOPE, OAuth2Properties.CATALOG_SCOPE); Map<String, String> optionalOAuthParams = OAuth2Util.buildOptionalParam(props); + if (!props.containsKey(OAuth2Properties.OAUTH2_SERVER_URI) + && (hasInitToken || hasCredential) + && !PropertyUtil.propertyAsBoolean(props, "rest.sigv4-enabled", false)) { + LOG.warn( + "Iceberg REST client is missing the OAuth2 server URI configuration and defaults to {}{}. " + + "This automatic fallback will be removed in a future Iceberg release." + + "It is recommended to configure the OAuth2 endpoint using the '{}' property to be prepared. " + + "This warning will disappear if the OAuth2 endpoint is explicitly configured. " + + "See https://github.com/apache/iceberg/issues/10537", + props.get(CatalogProperties.URI), + ResourcePaths.tokens(), + OAuth2Properties.OAUTH2_SERVER_URI); + } String oauth2ServerUri = props.getOrDefault(OAuth2Properties.OAUTH2_SERVER_URI, ResourcePaths.tokens()); try (RESTClient initClient = clientBuilder.apply(props)) { Map<String, String> initHeaders = RESTUtil.merge(configHeaders(props), OAuth2Util.authHeaders(initToken)); - if (credential != null && !credential.isEmpty()) { + if (hasCredential) { authResponse = OAuth2Util.fetchToken( initClient, initHeaders, credential, scope, oauth2ServerUri, optionalOAuthParams); @@ -253,10 +268,11 @@ public void initialize(String name, Map<String, String> unresolved) { this.io = newFileIO(SessionContext.createEmpty(), mergedProps); - this.fileIOCloser = newFileIOCloser(); + this.fileIOTracker = new FileIOTracker(); this.closeables = new CloseableGroup(); this.closeables.addCloseable(this.io); this.closeables.addCloseable(this.client); + this.closeables.addCloseable(fileIOTracker); this.closeables.setSuppressCloseFailure(true); this.snapshotMode = @@ -450,7 +466,7 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { private void trackFileIO(RESTTableOperations ops) { if (io != ops.io()) { - fileIOCloser.put(ops, ops.io()); + fileIOTracker.track(ops); } } @@ -532,7 +548,7 @@ public void createNamespace( public List<Namespace> listNamespaces(SessionContext context, Namespace namespace) { Map<String, String> queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { - queryParams.put("parent", RESTUtil.NAMESPACE_JOINER.join(namespace.levels())); + queryParams.put("parent", RESTUtil.encodeNamespace(namespace)); } ImmutableList.Builder<Namespace> namespaces = ImmutableList.builder(); @@ -626,11 +642,6 @@ public void close() throws IOException { if (closeables != null) { closeables.close(); } - - if (fileIOCloser != null) { - fileIOCloser.invalidateAll(); - fileIOCloser.cleanUp(); - } } private void shutdownRefreshExecutor() { @@ -1073,19 +1084,6 @@ private static Cache<String, AuthSession> newSessionCache(Map<String, String> pr .build(); } - private Cache<TableOperations, FileIO> newFileIOCloser() { - return Caffeine.newBuilder() - .weakKeys() - .removalListener( - (RemovalListener<TableOperations, FileIO>) - (ops, fileIO, cause) -> { - if (null != fileIO) { - fileIO.close(); - } - }) - .build(); - } - public void commitTransaction(SessionContext context, List<TableCommit> commits) { List<UpdateTableRequest> tableChanges = Lists.newArrayListWithCapacity(commits.size()); 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<String, String> baseHeaders, AuthConfig config) { this.config = config; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public AuthSession( - Map<String, String> 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<String, String> 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<?>, 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 d7221e7d4545..f86e5ddc5a8a 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -20,84 +20,121 @@ import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayDeque; +import java.util.Deque; import java.util.Iterator; import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import org.apache.iceberg.exceptions.RuntimeIOException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ParallelIterable<T> extends CloseableGroup implements CloseableIterable<T> { + + private static final Logger LOG = LoggerFactory.getLogger(ParallelIterable.class); + + // Logic behind default value: ParallelIterable is often used for file planning. + // Assuming that a DataFile or DeleteFile is about 500 bytes, a 30k limit uses 14.3 MB of memory. + private static final int DEFAULT_MAX_QUEUE_SIZE = 30_000; + private final Iterable<? extends Iterable<T>> iterables; private final ExecutorService workerPool; + // Bound for number of items in the queue to limit memory consumption + // even in the case when input iterables are large. + private final int approximateMaxQueueSize; + public ParallelIterable(Iterable<? extends Iterable<T>> iterables, ExecutorService workerPool) { - this.iterables = iterables; - this.workerPool = workerPool; + this(iterables, workerPool, DEFAULT_MAX_QUEUE_SIZE); + } + + public ParallelIterable( + Iterable<? extends Iterable<T>> iterables, + ExecutorService workerPool, + int approximateMaxQueueSize) { + this.iterables = Preconditions.checkNotNull(iterables, "Input iterables cannot be null"); + this.workerPool = Preconditions.checkNotNull(workerPool, "Worker pool cannot be null"); + this.approximateMaxQueueSize = approximateMaxQueueSize; } @Override public CloseableIterator<T> iterator() { - ParallelIterator<T> iter = new ParallelIterator<>(iterables, workerPool); + ParallelIterator<T> iter = + new ParallelIterator<>(iterables, workerPool, approximateMaxQueueSize); addCloseable(iter); return iter; } - private static class ParallelIterator<T> implements CloseableIterator<T> { - private final Iterator<Runnable> tasks; + @VisibleForTesting + static class ParallelIterator<T> implements CloseableIterator<T> { + private final Iterator<Task<T>> tasks; + private final Deque<Task<T>> yieldedTasks = new ArrayDeque<>(); private final ExecutorService workerPool; - private final Future<?>[] taskFutures; + private final CompletableFuture<Optional<Task<T>>>[] taskFutures; private final ConcurrentLinkedQueue<T> queue = new ConcurrentLinkedQueue<>(); - private volatile boolean closed = false; + private final int maxQueueSize; + private final AtomicBoolean closed = new AtomicBoolean(false); private ParallelIterator( - Iterable<? extends Iterable<T>> iterables, ExecutorService workerPool) { + Iterable<? extends Iterable<T>> iterables, ExecutorService workerPool, int maxQueueSize) { this.tasks = Iterables.transform( - iterables, - iterable -> - (Runnable) - () -> { - try (Closeable ignored = - (iterable instanceof Closeable) ? (Closeable) iterable : () -> {}) { - for (T item : iterable) { - // exit manually because `ConcurrentLinkedQueue` can't be - // interrupted - if (closed) { - return; - } - - queue.add(item); - } - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close iterable"); - } - }) + iterables, iterable -> new Task<>(iterable, queue, closed, maxQueueSize)) .iterator(); this.workerPool = workerPool; + Preconditions.checkArgument(maxQueueSize > 0, "Max queue size must be greater than 0"); + this.maxQueueSize = maxQueueSize; // submit 2 tasks per worker at a time - this.taskFutures = new Future[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; + this.taskFutures = new CompletableFuture[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; } @Override + @SuppressWarnings("FutureReturnValueIgnored") public void close() { // close first, avoid new task submit - this.closed = true; + this.closed.set(true); - // cancel background tasks - for (Future<?> taskFuture : taskFutures) { - if (taskFuture != null && !taskFuture.isDone()) { - taskFuture.cancel(true); + try (Closer closer = Closer.create()) { + synchronized (this) { + yieldedTasks.forEach(closer::register); + yieldedTasks.clear(); } + + // cancel background tasks and close continuations if any + for (CompletableFuture<Optional<Task<T>>> taskFuture : taskFutures) { + if (taskFuture != null) { + taskFuture.cancel(true); + taskFuture.thenAccept( + continuation -> { + if (continuation.isPresent()) { + try { + continuation.get().close(); + } catch (IOException e) { + LOG.error("Task close failed", e); + } + } + }); + } + } + + // clean queue + this.queue.clear(); + } catch (IOException e) { + throw new UncheckedIOException("Close failed", e); } - // clean queue - this.queue.clear(); } /** @@ -107,15 +144,17 @@ public void close() { * * @return true if there are pending tasks, false otherwise */ - private boolean checkTasks() { + private synchronized boolean checkTasks() { + Preconditions.checkState(!closed.get(), "Already closed"); boolean hasRunningTask = false; for (int i = 0; i < taskFutures.length; i += 1) { if (taskFutures[i] == null || taskFutures[i].isDone()) { if (taskFutures[i] != null) { - // check for task failure and re-throw any exception + // check for task failure and re-throw any exception. Enqueue continuation if any. try { - taskFutures[i].get(); + Optional<Task<T>> continuation = taskFutures[i].get(); + continuation.ifPresent(yieldedTasks::addLast); } catch (ExecutionException e) { if (e.getCause() instanceof RuntimeException) { // rethrow a runtime exception @@ -136,29 +175,29 @@ private boolean checkTasks() { } } - return !closed && (tasks.hasNext() || hasRunningTask); + return !closed.get() && (tasks.hasNext() || hasRunningTask); } - private Future<?> submitNextTask() { - if (!closed && tasks.hasNext()) { - return workerPool.submit(tasks.next()); + private CompletableFuture<Optional<Task<T>>> submitNextTask() { + if (!closed.get()) { + if (!yieldedTasks.isEmpty()) { + return CompletableFuture.supplyAsync(yieldedTasks.removeFirst(), workerPool); + } else if (tasks.hasNext()) { + return CompletableFuture.supplyAsync(tasks.next(), workerPool); + } } return null; } @Override public synchronized boolean hasNext() { - Preconditions.checkState(!closed, "Already closed"); - - // if the consumer is processing records more slowly than the producers, then this check will - // prevent tasks from being submitted. while the producers are running, this will always - // return here before running checkTasks. when enough of the tasks are finished that the - // consumer catches up, then lots of new tasks will be submitted at once. this behavior is - // okay because it ensures that records are not stacking up waiting to be consumed and taking - // up memory. - // - // consumers that process results quickly will periodically exhaust the queue and submit new - // tasks when checkTasks runs. fast consumers should not be delayed. + Preconditions.checkState(!closed.get(), "Already closed"); + + // If the consumer is processing records more slowly than the producers, the producers will + // eventually fill the queue and yield, returning continuations. Continuations and new tasks + // are started by checkTasks(). The check here prevents us from restarting continuations or + // starting new tasks before the queue is emptied. Restarting too early would lead to tasks + // yielding very quickly (CPU waste on scheduling). if (!queue.isEmpty()) { return true; } @@ -191,5 +230,84 @@ public synchronized T next() { } return queue.poll(); } + + @VisibleForTesting + int queueSize() { + return queue.size(); + } + } + + private static class Task<T> implements Supplier<Optional<Task<T>>>, Closeable { + private final Iterable<T> input; + private final ConcurrentLinkedQueue<T> queue; + private final AtomicBoolean closed; + private final int approximateMaxQueueSize; + + private Iterator<T> iterator = null; + + Task( + Iterable<T> input, + ConcurrentLinkedQueue<T> queue, + AtomicBoolean closed, + int approximateMaxQueueSize) { + this.input = Preconditions.checkNotNull(input, "input cannot be null"); + this.queue = Preconditions.checkNotNull(queue, "queue cannot be null"); + this.closed = Preconditions.checkNotNull(closed, "closed cannot be null"); + this.approximateMaxQueueSize = approximateMaxQueueSize; + } + + @Override + public Optional<Task<T>> get() { + try { + if (iterator == null) { + iterator = input.iterator(); + } + + while (iterator.hasNext()) { + if (queue.size() >= approximateMaxQueueSize) { + // Yield when queue is over the size limit. Task will be resubmitted later and continue + // the work. + return Optional.of(this); + } + + T next = iterator.next(); + if (closed.get()) { + break; + } + + queue.add(next); + } + } catch (Throwable e) { + try { + close(); + } catch (IOException closeException) { + // self-suppression is not permitted + // (e and closeException to be the same is unlikely, but possible) + if (closeException != e) { + e.addSuppressed(closeException); + } + } + + throw e; + } + + try { + close(); + } catch (IOException e) { + throw new UncheckedIOException("Close failed", e); + } + + // The task is complete. Returning empty means there is no continuation that should be + // executed. + return Optional.empty(); + } + + @Override + public void close() throws IOException { + iterator = null; + if (input instanceof Closeable) { + ((Closeable) input).close(); + } + } } } diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java index eff37fa5a9e8..184f38b32461 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java @@ -200,7 +200,7 @@ public String toString() { StringBuilder partitionStringBuilder = new StringBuilder(); partitionStringBuilder.append(structType.fields().get(i).name()); partitionStringBuilder.append("="); - partitionStringBuilder.append(s.get(i, Object.class).toString()); + partitionStringBuilder.append(s.get(i, Object.class)); partitionDataJoiner.add(partitionStringBuilder.toString()); } } diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java index 4a5f0c792baa..83735939317b 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java @@ -26,6 +26,7 @@ import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -98,4 +99,13 @@ public static StructLike coercePartition( projection.wrap(partition); return projection; } + + public static Map<Integer, PartitionSpec> indexSpecs(List<PartitionSpec> specs) { + ImmutableMap.Builder<Integer, PartitionSpec> builder = ImmutableMap.builder(); + for (PartitionSpec spec : specs) { + builder.put(spec.specId(), spec); + } + + return builder.build(); + } } diff --git a/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java b/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java index ddba0b0bbea4..216f55eae30c 100644 --- a/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java @@ -69,7 +69,7 @@ public static byte[] serializeToBytes( } } - @SuppressWarnings("unchecked") + @SuppressWarnings({"DangerousJavaDeserialization", "unchecked"}) public static <T> T deserializeFromBytes(byte[] bytes) { if (bytes == null) { return null; 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 <E extends Exception> void runTaskWithRetry(Task<I, E> 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/util/ZOrderByteUtils.java b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java index 4a5805742443..7edca59731a5 100644 --- a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java +++ b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java @@ -50,41 +50,45 @@ static ByteBuffer allocatePrimitiveBuffer() { return ByteBuffer.allocate(PRIMITIVE_BUFFER_SIZE); } - /** - * Signed ints do not have their bytes in magnitude order because of the sign bit. To fix this, - * flip the sign bit so that all negatives are ordered before positives. This essentially shifts - * the 0 value so that we don't break our ordering when we cross the new 0 value. - */ + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ public static ByteBuffer intToOrderedBytes(int val, ByteBuffer reuse) { - ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); - bytes.putLong(((long) val) ^ 0x8000000000000000L); - return bytes; + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); } /** - * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} + * Signed longs do not have their bytes in magnitude order because of the sign bit. To fix this, + * flip the sign bit so that all negatives are ordered before positives. This essentially shifts + * the 0 value so that we don't break our ordering when we cross the new 0 value. */ - public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { + public static ByteBuffer wholeNumberOrderedBytes(long val, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); bytes.putLong(val ^ 0x8000000000000000L); return bytes; } - /** - * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} - */ - public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { - return intToOrderedBytes(val, reuse); + /** Internally just calls {@link #floatingPointOrderedBytes(double, ByteBuffer)} */ + public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { + return floatingPointOrderedBytes(val, reuse); } - /** - * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} - */ - public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { - return intToOrderedBytes(val, reuse); + /** Internally just calls {@link #floatingPointOrderedBytes(double, ByteBuffer)} */ + public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) { + return floatingPointOrderedBytes(val, reuse); } /** @@ -92,15 +96,10 @@ public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { * y), they are ordered the same way when their bits are reinterpreted as sign-magnitude * integers.” * - * <p>Which means floats can be treated as sign magnitude integers which can then be converted + * <p>Which means doubles can be treated as sign magnitude integers which can then be converted * into lexicographically comparable bytes */ - public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { - return doubleToOrderedBytes(val, reuse); - } - - /** Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)} */ - public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) { + public static ByteBuffer floatingPointOrderedBytes(double val, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); long lval = Double.doubleToLongBits(val); lval ^= ((lval >> (Integer.SIZE - 1)) | Long.MIN_VALUE); 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..986e8608c082 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<Object> parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } static final DeleteFile FILE_A_POS_1 = @@ -139,6 +139,8 @@ public void testMinSequenceNumberFilteringForFiles() { DataFile file = unpartitionedFile(partSpec); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isFalse(); assertThat(index.forDataFile(0, file)).as("Only one delete file should apply").hasSize(1); } @@ -158,6 +160,9 @@ public void testUnpartitionedDeletes() { .specsById(ImmutableMap.of(partSpec.specId(), partSpec, 1, SPEC)) .build(); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isTrue(); + DataFile unpartitionedFile = unpartitionedFile(partSpec); assertThat(index.forDataFile(0, unpartitionedFile)) .as("All deletes should apply to seq 0") @@ -213,6 +218,9 @@ public void testPartitionedDeleteIndex() { .specsById(ImmutableMap.of(SPEC.specId(), SPEC, 1, PartitionSpec.unpartitioned())) .build(); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isTrue(); + assertThat(index.forDataFile(0, FILE_A)) .as("All deletes should apply to seq 0") .isEqualTo(deleteFiles); diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index 98a6eafaf8f6..e48f23ff9a0b 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -19,27 +19,48 @@ package org.apache.iceberg; import java.nio.ByteBuffer; +import java.util.Comparator; import java.util.Map; import java.util.Random; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.MetricsModes.Counts; +import org.apache.iceberg.MetricsModes.MetricsMode; +import org.apache.iceberg.MetricsModes.None; +import org.apache.iceberg.MetricsModes.Truncate; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.RandomUtil; public class FileGenerationUtil { private FileGenerationUtil() {} public static DataFile generateDataFile(Table table, StructLike partition) { + return generateDataFile(table, partition, ImmutableMap.of(), ImmutableMap.of()); + } + + public static DataFile generateDataFile( + Table table, + StructLike partition, + Map<Integer, ByteBuffer> lowerBounds, + Map<Integer, ByteBuffer> upperBounds) { Schema schema = table.schema(); PartitionSpec spec = table.spec(); LocationProvider locations = table.locationProvider(); String path = locations.newDataLocation(spec, partition, generateFileName()); long fileSize = generateFileSize(); - Metrics metrics = generateRandomMetrics(schema); + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + Metrics metrics = generateRandomMetrics(schema, metricsConfig, lowerBounds, upperBounds); return DataFiles.builder(spec) .withPath(path) .withPartition(partition) @@ -91,7 +112,11 @@ public static String generateFileName() { return String.format("%d-%d-%s-%d.parquet", partitionId, taskId, operationId, fileCount); } - public static Metrics generateRandomMetrics(Schema schema) { + public static Metrics generateRandomMetrics( + Schema schema, + MetricsConfig metricsConfig, + Map<Integer, ByteBuffer> knownLowerBounds, + Map<Integer, ByteBuffer> knownUpperBounds) { long rowCount = generateRowCount(); Map<Integer, Long> columnSizes = Maps.newHashMap(); Map<Integer, Long> valueCounts = Maps.newHashMap(); @@ -106,12 +131,16 @@ public static Metrics generateRandomMetrics(Schema schema) { valueCounts.put(fieldId, generateValueCount()); nullValueCounts.put(fieldId, (long) random().nextInt(5)); nanValueCounts.put(fieldId, (long) random().nextInt(5)); - byte[] lower = new byte[16]; - random().nextBytes(lower); - lowerBounds.put(fieldId, ByteBuffer.wrap(lower)); - byte[] upper = new byte[16]; - random().nextBytes(upper); - upperBounds.put(fieldId, ByteBuffer.wrap(upper)); + if (knownLowerBounds.containsKey(fieldId) && knownUpperBounds.containsKey(fieldId)) { + lowerBounds.put(fieldId, knownLowerBounds.get(fieldId)); + upperBounds.put(fieldId, knownUpperBounds.get(fieldId)); + } else if (column.type().isPrimitiveType()) { + PrimitiveType type = column.type().asPrimitiveType(); + MetricsMode metricsMode = metricsConfig.columnMode(column.name()); + Pair<ByteBuffer, ByteBuffer> bounds = generateBounds(type, metricsMode); + lowerBounds.put(fieldId, bounds.first()); + upperBounds.put(fieldId, bounds.second()); + } } return new Metrics( @@ -185,6 +214,37 @@ private static long generateFileSize() { return random().nextInt(50_000); } + private static Pair<ByteBuffer, ByteBuffer> generateBounds(PrimitiveType type, MetricsMode mode) { + Comparator<Object> cmp = Comparators.forType(type); + Object value1 = generateBound(type, mode); + Object value2 = generateBound(type, mode); + if (cmp.compare(value1, value2) > 0) { + ByteBuffer lowerBuffer = Conversions.toByteBuffer(type, value2); + ByteBuffer upperBuffer = Conversions.toByteBuffer(type, value1); + return Pair.of(lowerBuffer, upperBuffer); + } else { + ByteBuffer lowerBuffer = Conversions.toByteBuffer(type, value1); + ByteBuffer upperBuffer = Conversions.toByteBuffer(type, value2); + return Pair.of(lowerBuffer, upperBuffer); + } + } + + private static Object generateBound(PrimitiveType type, MetricsMode mode) { + if (mode instanceof None || mode instanceof Counts) { + return null; + } else if (mode instanceof Truncate) { + Object value = RandomUtil.generatePrimitive(type, random()); + Transform<Object, Object> truncate = Transforms.truncate(((Truncate) mode).length()); + if (truncate.canTransform(type)) { + return truncate.bind(type).apply(value); + } else { + return value; + } + } else { + return RandomUtil.generatePrimitive(type, random()); + } + } + private static Random random() { return ThreadLocalRandom.current(); } diff --git a/core/src/test/java/org/apache/iceberg/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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } protected Set<String> 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<Object> 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/TestAllManifestsTableTaskParser.java b/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java new file mode 100644 index 000000000000..2f057d7bd5a8 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.junit.jupiter.api.Test; + +public class TestAllManifestsTableTaskParser { + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid manifest task: null"); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.toJson(createTask(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for manifest task: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest task: non-object "); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest task: non-object "); + } + + @Test + public void testParser() { + AllManifestsTable.ManifestListReadTask task = createTask(); + String jsonStr = ScanTaskParser.toJson(task); + assertThat(jsonStr).isEqualTo(taskJson()); + AllManifestsTable.ManifestListReadTask deserializedTask = + (AllManifestsTable.ManifestListReadTask) ScanTaskParser.fromJson(jsonStr, false); + assertTaskEquals(task, deserializedTask); + } + + private AllManifestsTable.ManifestListReadTask createTask() { + Schema dataTableSchema = TestBase.SCHEMA; + HadoopFileIO fileIO = new HadoopFileIO(); + fileIO.initialize(ImmutableMap.of("k1", "v1", "k2", "v2")); + Map<Integer, PartitionSpec> specsById = + PartitionUtil.indexSpecs( + Arrays.asList(PartitionSpec.builderFor(dataTableSchema).bucket("data", 16).build())); + + return new AllManifestsTable.ManifestListReadTask( + dataTableSchema, + fileIO, + AllManifestsTable.MANIFEST_FILE_SCHEMA, + specsById, + "/path/manifest-list-file.avro", + Expressions.equal("id", 1), + 1L); + } + + private String taskJson() { + return "{\"task-type\":\"all-manifests-table-task\"," + + "\"data-table-schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"file-io\":{\"io-impl\":\"org.apache.iceberg.hadoop.HadoopFileIO\"," + + "\"properties\":{\"k1\":\"v1\",\"k2\":\"v2\"}}," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{" + + "\"id\":14,\"name\":\"content\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":1,\"name\":\"path\",\"required\":true,\"type\":\"string\"}," + + "{\"id\":2,\"name\":\"length\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":3,\"name\":\"partition_spec_id\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"added_snapshot_id\",\"required\":false,\"type\":\"long\"}," + + "{\"id\":5,\"name\":\"added_data_files_count\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":6,\"name\":\"existing_data_files_count\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":7,\"name\":\"deleted_data_files_count\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":15,\"name\":\"added_delete_files_count\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":16,\"name\":\"existing_delete_files_count\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":17,\"name\":\"deleted_delete_files_count\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":8,\"name\":\"partition_summaries\",\"required\":false,\"type\":" + + "{\"type\":\"list\",\"element-id\":9,\"element\":{\"type\":\"struct\",\"fields\":[{" + + "\"id\":10,\"name\":\"contains_null\",\"required\":true,\"type\":\"boolean\"}," + + "{\"id\":11,\"name\":\"contains_nan\",\"required\":true,\"type\":\"boolean\"}," + + "{\"id\":12,\"name\":\"lower_bound\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":13,\"name\":\"upper_bound\",\"required\":false,\"type\":\"string\"}]},\"element-required\":true}}," + + "{\"id\":18,\"name\":\"reference_snapshot_id\",\"required\":true,\"type\":\"long\"}]}," + + "\"partition-specs\":[{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}]," + + "\"manifest-list-Location\":\"/path/manifest-list-file.avro\"," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}," + + "\"reference-snapshot-id\":1}"; + } + + private void assertTaskEquals( + AllManifestsTable.ManifestListReadTask expected, + AllManifestsTable.ManifestListReadTask actual) { + + HadoopFileIO expectedIO = (HadoopFileIO) expected.io(); + HadoopFileIO actualIO = (HadoopFileIO) expected.io(); + assertThat(actualIO.properties()).isEqualTo(expectedIO.properties()); + + assertThat(actual.dataTableSchema().asStruct()) + .as("Data table schema should match") + .isEqualTo(expected.dataTableSchema().asStruct()); + assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); + + assertThat(actual.specsById()).isEqualTo(expected.specsById()); + assertThat(actual.manifestListLocation()).isEqualTo(expected.manifestListLocation()); + assertThat(actual.residual().toString()).isEqualTo(expected.residual().toString()); + assertThat(actual.referenceSnapshotId()).isEqualTo(expected.referenceSnapshotId()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index bbba192fab7f..41b301668722 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -37,7 +37,7 @@ public class TestCommitReporting extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 0c6b50b37792..766dd85a0655 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -40,7 +40,7 @@ public class TestCreateTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> 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<Object> 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<Object> 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<ManifestFile> 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/TestFileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java new file mode 100644 index 000000000000..ea44aa73c6d6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import org.apache.iceberg.MetricsModes.MetricsMode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.junit.jupiter.api.Test; + +public class TestFileGenerationUtil { + + public static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + + @Test + public void testBoundsWithDefaultMetricsConfig() { + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + Metrics metrics = + FileGenerationUtil.generateRandomMetrics( + SCHEMA, + metricsConfig, + ImmutableMap.of() /* no known lower bounds */, + ImmutableMap.of() /* no known upper bounds */); + + assertThat(metrics.lowerBounds()).hasSize(SCHEMA.columns().size()); + assertThat(metrics.upperBounds()).hasSize(SCHEMA.columns().size()); + + checkBounds(metrics, metricsConfig); + } + + @Test + public void testBoundsWithSpecificValues() { + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + NestedField intField = SCHEMA.findField("int_col"); + PrimitiveType type = intField.type().asPrimitiveType(); + ByteBuffer intLower = Conversions.toByteBuffer(type, 0); + ByteBuffer intUpper = Conversions.toByteBuffer(type, Integer.MAX_VALUE); + Metrics metrics = + FileGenerationUtil.generateRandomMetrics( + SCHEMA, + metricsConfig, + ImmutableMap.of(intField.fieldId(), intLower), + ImmutableMap.of(intField.fieldId(), intUpper)); + + assertThat(metrics.lowerBounds()).hasSize(SCHEMA.columns().size()); + assertThat(metrics.upperBounds()).hasSize(SCHEMA.columns().size()); + + checkBounds(metrics, metricsConfig); + + ByteBuffer actualIntLower = metrics.lowerBounds().get(intField.fieldId()); + ByteBuffer actualIntUpper = metrics.upperBounds().get(intField.fieldId()); + assertThat(actualIntLower).isEqualTo(intLower); + assertThat(actualIntUpper).isEqualTo(intUpper); + } + + private void checkBounds(Metrics metrics, MetricsConfig metricsConfig) { + for (NestedField field : SCHEMA.columns()) { + MetricsMode mode = metricsConfig.columnMode(field.name()); + ByteBuffer lowerBuffer = metrics.lowerBounds().get(field.fieldId()); + ByteBuffer upperBuffer = metrics.upperBounds().get(field.fieldId()); + if (mode.equals(MetricsModes.None.get()) || mode.equals(MetricsModes.Counts.get())) { + assertThat(lowerBuffer).isNull(); + assertThat(upperBuffer).isNull(); + } else { + checkBounds(field.type().asPrimitiveType(), lowerBuffer, upperBuffer); + } + } + } + + private void checkBounds(PrimitiveType type, ByteBuffer lowerBuffer, ByteBuffer upperBuffer) { + Object lower = Conversions.fromByteBuffer(type, lowerBuffer); + Object upper = Conversions.fromByteBuffer(type, upperBuffer); + Comparator<Object> cmp = Comparators.forType(type); + assertThat(cmp.compare(lower, upper)).isLessThanOrEqualTo(0); + } +} 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/TestFilesTableTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFilesTableTaskParser.java new file mode 100644 index 000000000000..bea60601377e --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestFilesTableTaskParser.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.junit.jupiter.api.Test; + +public class TestFilesTableTaskParser { + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> FilesTableTaskParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid files task: null"); + + assertThatThrownBy(() -> FilesTableTaskParser.toJson(createTask(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> FilesTableTaskParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for files task: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> FilesTableTaskParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for files task: non-object "); + + assertThatThrownBy(() -> FilesTableTaskParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for files task: non-object "); + } + + @Test + public void testParser() { + BaseFilesTable.ManifestReadTask task = createTask(); + String jsonStr = ScanTaskParser.toJson(task); + assertThat(jsonStr).isEqualTo(taskJson()); + BaseFilesTable.ManifestReadTask deserializedTask = + (BaseFilesTable.ManifestReadTask) ScanTaskParser.fromJson(jsonStr, false); + assertTaskEquals(task, deserializedTask); + } + + private BaseFilesTable.ManifestReadTask createTask() { + Schema schema = TestBase.SCHEMA; + HadoopFileIO fileIO = new HadoopFileIO(); + fileIO.initialize(ImmutableMap.of("k1", "v1", "k2", "v2")); + Map<Integer, PartitionSpec> specsById = + PartitionUtil.indexSpecs( + Arrays.asList(PartitionSpec.builderFor(schema).bucket("data", 16).build())); + ManifestFile manifestFile = TestManifestFileParser.createManifestFile(); + return new BaseFilesTable.ManifestReadTask( + schema, fileIO, specsById, manifestFile, schema, Expressions.equal("id", 1)); + } + + private String taskJson() { + return "{\"task-type\":\"files-table-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{" + + "\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{" + + "\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"file-io\":{\"io-impl\":\"org.apache.iceberg.hadoop.HadoopFileIO\"," + + "\"properties\":{\"k1\":\"v1\",\"k2\":\"v2\"}}," + + "\"partition-specs\":[{\"spec-id\":0,\"fields\":[{" + + "\"name\":\"data_bucket\",\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}]," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}," + + "\"manifest-file\":{\"path\":\"/path/input.m0.avro\"," + + "\"length\":5878,\"partition-spec-id\":0,\"content\":0,\"sequence-number\":1,\"min-sequence-number\":2," + + "\"added-snapshot-id\":12345678901234567," + + "\"added-files-count\":1,\"existing-files-count\":3,\"deleted-files-count\":0," + + "\"added-rows-count\":10,\"existing-rows-count\":30,\"deleted-rows-count\":0," + + "\"partition-field-summary\":[{\"contains-null\":true,\"contains-nan\":false," + + "\"lower-bound\":\"0A000000\",\"upper-bound\":\"64000000\"}],\"key-metadata\":\"DB030000\"}}"; + } + + private void assertTaskEquals( + BaseFilesTable.ManifestReadTask expected, BaseFilesTable.ManifestReadTask actual) { + assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); + assertThat(actual.projection().asStruct()) + .as("Projected schema should match") + .isEqualTo(expected.projection().asStruct()); + + HadoopFileIO expectedIO = (HadoopFileIO) expected.io(); + HadoopFileIO actualIO = (HadoopFileIO) expected.io(); + assertThat(actualIO.properties()).isEqualTo(expectedIO.properties()); + + assertThat(actual.specsById()).isEqualTo(expected.specsById()); + assertThat(actual.residual().toString()).isEqualTo(expected.residual().toString()); + assertThat(actual.manifest()).isEqualTo(expected.manifest()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/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<Object> 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..3414f1858e98 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -28,51 +28,83 @@ public class TestFormatVersions extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> parameters() { - return Arrays.asList(1); + return Arrays.asList(1, 2); } @TestTemplate public void testDefaultFormatVersion() { - assertThat(table.ops().current().formatVersion()).isEqualTo(1); + assertThat(table.ops().current().formatVersion()).isEqualTo(formatVersion); } @TestTemplate public void testFormatVersionUpgrade() { TableOperations ops = table.ops(); - TableMetadata base = ops.current(); - ops.commit(base, base.upgradeToFormatVersion(2)); + int newFormatVersion = formatVersion + 1; + + TableMetadata newTableMetadata = ops.current().upgradeToFormatVersion(newFormatVersion); + + assertThat( + newTableMetadata.changes().stream() + .filter(MetadataUpdate.UpgradeFormatVersion.class::isInstance) + .map(MetadataUpdate.UpgradeFormatVersion.class::cast) + .map(MetadataUpdate.UpgradeFormatVersion::formatVersion)) + .containsExactly(newFormatVersion); + + ops.commit(ops.current(), newTableMetadata); - assertThat(ops.current().formatVersion()).isEqualTo(2); + assertThat(ops.current().formatVersion()).isEqualTo(newFormatVersion); + } + + @TestTemplate + public void testFormatVersionUpgradeToLatest() { + TableOperations ops = table.ops(); + + TableMetadata newTableMetadata = + ops.current().upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION); + + assertThat( + newTableMetadata.changes().stream() + .filter(MetadataUpdate.UpgradeFormatVersion.class::isInstance) + .map(MetadataUpdate.UpgradeFormatVersion.class::cast) + .map(MetadataUpdate.UpgradeFormatVersion::formatVersion)) + .isEqualTo(List.of(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION)); + + ops.commit(ops.current(), newTableMetadata); + + assertThat(ops.current().formatVersion()) + .isEqualTo(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION); } @TestTemplate public void testFormatVersionDowngrade() { TableOperations ops = table.ops(); - TableMetadata base = ops.current(); - ops.commit(base, base.upgradeToFormatVersion(2)); + int newFormatVersion = formatVersion + 1; + ops.commit(ops.current(), ops.current().upgradeToFormatVersion(newFormatVersion)); - assertThat(ops.current().formatVersion()).isEqualTo(2); + assertThat(ops.current().formatVersion()).isEqualTo(newFormatVersion); - assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(1)) + assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(formatVersion)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot downgrade v2 table to v1"); + .hasMessage( + String.format("Cannot downgrade v%d table to v%d", newFormatVersion, formatVersion)); - assertThat(ops.current().formatVersion()).isEqualTo(2); + assertThat(ops.current().formatVersion()).isEqualTo(newFormatVersion); } @TestTemplate public void testFormatVersionUpgradeNotSupported() { TableOperations ops = table.ops(); TableMetadata base = ops.current(); + int unsupportedFormatVersion = TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1; assertThatThrownBy( - () -> - ops.commit( - base, - base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))) + () -> ops.commit(base, base.upgradeToFormatVersion(unsupportedFormatVersion))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot upgrade table to unsupported format version: v3 (supported: v2)"); + .hasMessage( + String.format( + "Cannot upgrade table to unsupported format version: v%d (supported: v%d)", + unsupportedFormatVersion, TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION)); - assertThat(ops.current().formatVersion()).isEqualTo(1); + assertThat(ops.current().formatVersion()).isEqualTo(formatVersion); } } diff --git a/core/src/test/java/org/apache/iceberg/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<Object> 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<Object> 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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestManifestFileParser.java b/core/src/test/java/org/apache/iceberg/TestManifestFileParser.java new file mode 100644 index 000000000000..5a6e99c984f0 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestManifestFileParser.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.junit.jupiter.api.Test; + +public class TestManifestFileParser { + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> ManifestFileParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid manifest file: null"); + + assertThatThrownBy(() -> ManifestFileParser.toJson(createManifestFile(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> ManifestFileParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for manifest file: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> ManifestFileParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest file: non-object "); + + assertThatThrownBy(() -> ManifestFileParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest file: non-object "); + } + + @Test + public void testParser() throws Exception { + ManifestFile manifest = createManifestFile(); + String jsonStr = JsonUtil.generate(gen -> ManifestFileParser.toJson(manifest, gen), false); + assertThat(jsonStr).isEqualTo(manifestFileJson()); + } + + static ManifestFile createManifestFile() { + ByteBuffer lowerBound = Conversions.toByteBuffer(Types.IntegerType.get(), 10); + ByteBuffer upperBound = Conversions.toByteBuffer(Types.IntegerType.get(), 100); + List<ManifestFile.PartitionFieldSummary> partitionFieldSummaries = + Arrays.asList(new GenericPartitionFieldSummary(true, false, lowerBound, upperBound)); + ByteBuffer keyMetadata = Conversions.toByteBuffer(Types.IntegerType.get(), 987); + + return new GenericManifestFile( + "/path/input.m0.avro", + 5878L, + 0, + ManifestContent.DATA, + 1L, + 2L, + 12345678901234567L, + 1, + 10L, + 3, + 30L, + 0, + 0L, + partitionFieldSummaries, + keyMetadata); + } + + private String manifestFileJson() { + return "{\"path\":\"/path/input.m0.avro\"," + + "\"length\":5878,\"partition-spec-id\":0,\"content\":0,\"sequence-number\":1,\"min-sequence-number\":2," + + "\"added-snapshot-id\":12345678901234567,\"added-files-count\":1,\"existing-files-count\":3,\"deleted-files-count\":0," + + "\"added-rows-count\":10,\"existing-rows-count\":30,\"deleted-rows-count\":0," + + "\"partition-field-summary\":[{\"contains-null\":true,\"contains-nan\":false," + + "\"lower-bound\":\"0A000000\",\"upper-bound\":\"64000000\"}]," + + "\"key-metadata\":\"DB030000\"}"; + } +} 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final Map<Integer, Long> 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<Object> 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/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 4719923e72ac..abfcb318334b 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -27,12 +27,14 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; @@ -90,6 +92,92 @@ public void testEmptyTableAppend() { statuses(Status.ADDED, Status.ADDED)); } + @TestTemplate + public void testEmptyTableAppendFilesWithDifferentSpecs() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + TableMetadata base = readMetadata(); + assertThat(base.currentSnapshot()).as("Should not have a current snapshot").isNull(); + assertThat(base.lastSequenceNumber()).as("Last sequence number should be 0").isEqualTo(0); + + table.updateSpec().addField("id").commit(); + PartitionSpec newSpec = table.spec(); + + assertThat(table.specs()).as("Table should have 2 specs").hasSize(2); + + DataFile fileNewSpec = + DataFiles.builder(newSpec) + .withPath("/path/to/data-b.parquet") + .withPartitionPath("data_bucket=0/id=0") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + Snapshot committedSnapshot = + commit(table, table.newAppend().appendFile(FILE_A).appendFile(fileNewSpec), branch); + + assertThat(committedSnapshot).as("Should create a snapshot").isNotNull(); + V1Assert.assertEquals( + "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); + V2Assert.assertEquals( + "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + + assertThat(committedSnapshot.allManifests(table.io())) + .as("Should create 2 manifests for initial write, 1 manifest per spec") + .hasSize(2); + + long snapshotId = committedSnapshot.snapshotId(); + + ImmutableMap<Integer, DataFile> expectedFileBySpec = + ImmutableMap.of(SPEC.specId(), FILE_A, newSpec.specId(), fileNewSpec); + + expectedFileBySpec.forEach( + (specId, expectedDataFile) -> { + ManifestFile manifestFileForSpecId = + committedSnapshot.allManifests(table.io()).stream() + .filter(m -> Objects.equals(m.partitionSpecId(), specId)) + .findAny() + .get(); + + validateManifest( + manifestFileForSpecId, + dataSeqs(1L), + fileSeqs(1L), + ids(snapshotId), + files(expectedDataFile), + statuses(Status.ADDED)); + }); + } + + @TestTemplate + public void testDataSpecThrowsExceptionIfDataFilesWithDifferentSpecsAreAdded() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + TableMetadata base = readMetadata(); + assertThat(base.currentSnapshot()).as("Should not have a current snapshot").isNull(); + assertThat(base.lastSequenceNumber()).as("Last sequence number should be 0").isEqualTo(0); + + table.updateSpec().addField("id").commit(); + PartitionSpec newSpec = table.spec(); + + assertThat(table.specs()).as("Table should have 2 specs").hasSize(2); + + DataFile fileNewSpec = + DataFiles.builder(newSpec) + .withPath("/path/to/data-b.parquet") + .withPartitionPath("data_bucket=0/id=0") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + MergeAppend mergeAppend = + (MergeAppend) table.newAppend().appendFile(FILE_A).appendFile(fileNewSpec); + assertThatThrownBy(mergeAppend::dataSpec) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot return a single partition spec: data files with different partition specs have been added"); + } + @TestTemplate public void testEmptyTableAppendManifest() throws IOException { assertThat(listManifestFiles()).isEmpty(); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index fadaeb079330..bf6456e85aec 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -37,10 +37,10 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestMetadataTableFilters extends TestBase { - private static final Set<MetadataTableType> aggFileTables = + private static final Set<MetadataTableType> 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: @@ -149,7 +149,7 @@ private int expectedScanTaskCount(int partitions) { } private boolean isAggFileTable(MetadataTableType tableType) { - return aggFileTables.contains(tableType); + return AGG_FILE_TABLES.contains(tableType); } private String partitionColumn(String colName) { @@ -317,7 +317,7 @@ public void testPartitionSpecEvolutionRemovalV1() { .withPartition(data10Key) .build(); PartitionKey data11Key = new PartitionKey(newSpec, table.schema()); - data10Key.set(1, 11); + data11Key.set(1, 11); DataFile data11 = DataFiles.builder(newSpec) .withPath("/path/to/data-11.parquet") @@ -465,8 +465,8 @@ public void testPartitionSpecEvolutionAdditiveV1() { .withPartition(data10Key) .build(); PartitionKey data11Key = new PartitionKey(newSpec, table.schema()); - data11Key.set(0, 1); // data=0 - data10Key.set(1, 11); // id=11 + data11Key.set(0, 1); // data=1 + data11Key.set(1, 11); // id=11 DataFile data11 = DataFiles.builder(newSpec) .withPath("/path/to/data-11.parquet") diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 0a3040939c04..146b859bef00 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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(); @@ -940,8 +939,8 @@ public void testPartitionSpecEvolutionAdditive() { .withPartition(data10Key) .build(); PartitionKey data11Key = new PartitionKey(newSpec, table.schema()); - data11Key.set(0, 1); // data=0 - data10Key.set(1, 11); // id=11 + data11Key.set(0, 1); // data=1 + data11Key.set(1, 11); // id=11 DataFile data11 = DataFiles.builder(newSpec) .withPath("/path/to/data-11.parquet") @@ -960,7 +959,7 @@ public void testPartitionSpecEvolutionAdditive() { TableScan scan = metadataTable.newScan().filter(filter); CloseableIterable<ManifestEntry<?>> 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<ManifestEntry<?>> 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<ManifestEntry<?>> 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()); @@ -1753,4 +1751,36 @@ public void testPositionDeletesManyColumns() { assertThat(scanTasks.get(0).file().path().toString()).isEqualTo("/path/to/delete1.parquet"); assertThat(scanTasks.get(1).file().path().toString()).isEqualTo("/path/to/delete2.parquet"); } + + @TestTemplate + public void testFilesTableEstimateSize() throws Exception { + preparePartitionedTable(true); + + assertEstimatedRowCount(new DataFilesTable(table), 4); + assertEstimatedRowCount(new AllDataFilesTable(table), 4); + assertEstimatedRowCount(new AllFilesTable(table), 4); + + if (formatVersion == 2) { + assertEstimatedRowCount(new DeleteFilesTable(table), 4); + assertEstimatedRowCount(new AllDeleteFilesTable(table), 4); + } + } + + @TestTemplate + public void testEntriesTableEstimateSize() throws Exception { + preparePartitionedTable(true); + + assertEstimatedRowCount(new ManifestEntriesTable(table), 4); + assertEstimatedRowCount(new AllEntriesTable(table), 4); + } + + private void assertEstimatedRowCount(Table metadataTable, int size) throws Exception { + TableScan scan = metadataTable.newScan(); + + try (CloseableIterable<FileScanTask> tasks = scan.planFiles()) { + List<FileScanTask> taskList = Lists.newArrayList(tasks); + assertThat(taskList.size()).isGreaterThan(0); + taskList.forEach(task -> assertThat(task.estimatedRowsCount()).isEqualTo(size)); + } + } } 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<Object> 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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java new file mode 100644 index 000000000000..349912bd78df --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java @@ -0,0 +1,873 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestPartitionSpecBuilderCaseSensitivity { + + private static final int V2_FORMAT_VERSION = 2; + private static final Schema SCHEMA_WITHOUT_NAME_CONFLICTS = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get()), + required(4, "order_date", Types.DateType.get()), + required(5, "order_time", Types.TimestampType.withoutZone()), + required(6, "ship_date", Types.DateType.get()), + required(7, "ship_time", Types.TimestampType.withoutZone())); + + private static final Schema SCHEMA_WITH_NAME_CONFLICTS = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get()), + required(4, "order_date", Types.DateType.get()), + required(5, "ORDER_DATE", Types.DateType.get()), + required(6, "order_time", Types.TimestampType.withoutZone()), + required(7, "ORDER_TIME", Types.TimestampType.withoutZone())); + + @TempDir private Path temp; + private File tableDir = null; + + @BeforeEach + public void setupTableDir() throws IOException { + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); + } + + @AfterEach + public void cleanupTables() { + TestTables.clearTables(); + } + + @Test + public void testPartitionTypeWithColumnNamesThatDifferOnlyInLetterCase() { + Schema schema = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get()), + required(4, "order_date", Types.DateType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").identity("DATA").build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "data", Types.StringType.get()), + NestedField.optional(1001, "DATA", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testPartitionTypeWithIdentityTargetName() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .identity("data", "partition1") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "partition1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .bucket("data", 10, "partition1") + .bucket("data", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).bucket("data", 10).build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_bucket", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("DATA", 10) + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_bucket", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("DATA", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("category", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("category", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testBucketTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .bucket("data", 10, "partition1") + .bucket("DATA", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testTruncateTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).truncate("data", 10).build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_trunc", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("DATA", 10) + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_trunc", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .truncate("data", 10, "partition1") + .truncate("data", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("DATA", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("category", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("category", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testTruncateTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .truncate("data", 10, "partition1") + .truncate("DATA", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testIdentityTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).identity("data").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentityTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("DATA") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentitySourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .identity("data", "partition1") + .identity("data", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: identity(2) conflicts with 1001: PARTITION1: identity(2)"); + } + + @Test + public void testIdentitySourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("DATA", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: identity(2) conflicts with 1001: PARTITION1: identity(2)"); + } + + @Test + public void testIdentityTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("category", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentityTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("category", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testIdentityTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .identity("data", "partition1") + .identity("DATA", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testAlwaysNullTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).alwaysNull("data").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_null", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("DATA") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_null", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .alwaysNull("data", "partition1") + .alwaysNull("data", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("DATA", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("category", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("category", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testAlwaysNullTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .alwaysNull("data", "partition1") + .alwaysNull("DATA", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testYearTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).year("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_year", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_year", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .year("order_date", "partition1") + .year("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: year(4) conflicts with 1001: PARTITION1: year(4)"); + } + + @Test + public void testYearSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: year(4) conflicts with 1001: PARTITION1: year(4)"); + } + + @Test + public void testYearTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testYearTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .year("order_date", "partition1") + .year("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testMonthTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).month("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_month", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_month", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .month("order_date", "partition1") + .month("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: month(4) conflicts with 1001: PARTITION1: month(4)"); + } + + @Test + public void testMonthSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: month(4) conflicts with 1001: PARTITION1: month(4)"); + } + + @Test + public void testMonthTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testMonthTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .month("order_date", "partition1") + .month("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testDayTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).day("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_day", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDayTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_day", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDaySourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .day("order_date", "partition1") + .day("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: day(4) conflicts with 1001: PARTITION1: day(4)"); + } + + @Test + public void testDaySourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: day(4) conflicts with 1001: PARTITION1: day(4)"); + } + + @Test + public void testDayTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.DateType.get()), + NestedField.optional(1001, "PARTITION1", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDayTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testDayTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .day("order_date", "partition1") + .day("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testHourTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).hour("order_time").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_time_hour", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("ORDER_TIME") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_time_hour", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .hour("order_time", "partition1") + .hour("order_time", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: hour(6) conflicts with 1001: PARTITION1: hour(6)"); + } + + @Test + public void testHourSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ORDER_TIME", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: hour(5) conflicts with 1001: PARTITION1: hour(5)"); + } + + @Test + public void testHourTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ship_time", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ship_time", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testHourTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .hour("order_time", "partition1") + .hour("ORDER_TIME", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index ff54929504ee..f5636a77be61 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.entry; import java.io.File; @@ -47,7 +48,7 @@ public class TestPartitionSpecInfo { @Parameters(name = "formatVersion = {0}") protected static List<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; @@ -95,6 +96,30 @@ public void testSpecInfoPartitionedTable() { .doesNotContainKey(Integer.MAX_VALUE); } + @TestTemplate + public void testSpecInfoPartitionedTableCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(schema).caseSensitive(false).identity("DATA").build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); + + assertThat(table.spec()).isEqualTo(spec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(spec.lastAssignedFieldId()); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec)) + .doesNotContainKey(Integer.MAX_VALUE); + } + + @TestTemplate + public void testSpecInfoPartitionedTableCaseSensitiveFails() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> { + PartitionSpec spec = + PartitionSpec.builderFor(schema).caseSensitive(true).identity("DATA").build(); + }) + .withMessage("Cannot find source column: DATA"); + } + @TestTemplate public void testColumnDropWithPartitionSpecEvolution() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitioning.java b/core/src/test/java/org/apache/iceberg/TestPartitioning.java index 91f0fe95c2fa..a4df125f1de2 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitioning.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitioning.java @@ -122,6 +122,25 @@ public void testPartitionTypeWithRenamesInV1Table() { assertThat(actualType).isEqualTo(expectedType); } + @Test + public void testPartitionTypeWithRenamesInV1TableCaseInsensitive() { + PartitionSpec initialSpec = + PartitionSpec.builderFor(SCHEMA).caseSensitive(false).identity("DATA", "p1").build(); + TestTables.TestTable table = + TestTables.create(tableDir, "test", SCHEMA, initialSpec, V1_FORMAT_VERSION); + + table.updateSpec().addField("category").commit(); + + table.updateSpec().renameField("p1", "p2").commit(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "p2", Types.StringType.get()), + NestedField.optional(1001, "category", Types.StringType.get())); + StructType actualType = Partitioning.partitionType(table); + assertThat(actualType).isEqualTo(expectedType); + } + @Test public void testPartitionTypeWithAddingBackSamePartitionFieldInV1Table() { TestTables.TestTable table = @@ -252,6 +271,23 @@ public void testGroupingKeyTypeWithRenamesInV1Table() { assertThat(actualType).isEqualTo(expectedType); } + @Test + public void testGroupingKeyTypeWithRenamesInV1TableCaseInsensitive() { + PartitionSpec initialSpec = + PartitionSpec.builderFor(SCHEMA).caseSensitive(false).identity("DATA", "p1").build(); + TestTables.TestTable table = + TestTables.create(tableDir, "test", SCHEMA, initialSpec, V1_FORMAT_VERSION); + + table.updateSpec().addField("category").commit(); + + table.updateSpec().renameField("p1", "p2").commit(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "p2", Types.StringType.get())); + StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); + assertThat(actualType).isEqualTo(expectedType); + } + @Test public void testGroupingKeyTypeWithRenamesInV2Table() { PartitionSpec initialSpec = PartitionSpec.builderFor(SCHEMA).identity("data", "p1").build(); diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index a25920a1d733..6fa77ae05c9a 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -67,6 +67,34 @@ public class TestReplacePartitions extends TestBase { .withRecordCount(1) .build(); + static final DataFile FILE_NULL_PARTITION = + DataFiles.builder(SPEC) + .withPath("/path/to/data-null-partition.parquet") + .withFileSizeInBytes(0) + .withPartitionPath("data_bucket=__HIVE_DEFAULT_PARTITION__") + .withRecordCount(0) + .build(); + + // Partition spec with VOID partition transform ("alwaysNull" in Java code.) + static final PartitionSpec SPEC_VOID = + PartitionSpec.builderFor(SCHEMA).alwaysNull("id").bucket("data", BUCKETS_NUMBER).build(); + + static final DataFile FILE_A_VOID_PARTITION = + DataFiles.builder(SPEC_VOID) + .withPath("/path/to/data-a-void-partition.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("id_null=__HIVE_DEFAULT_PARTITION__/data_bucket=0") + .withRecordCount(1) + .build(); + + static final DataFile FILE_B_VOID_PARTITION = + DataFiles.builder(SPEC_VOID) + .withPath("/path/to/data-b-void-partition.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("id_null=__HIVE_DEFAULT_PARTITION__/data_bucket=1") + .withRecordCount(10) + .build(); + static final DeleteFile FILE_UNPARTITIONED_A_DELETES = FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) .ofPositionDeletes() @@ -317,6 +345,55 @@ public void testValidateWithDefaultSnapshotId() { + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]"); } + @TestTemplate + public void testValidateWithNullPartition() { + commit(table, table.newReplacePartitions().addFile(FILE_NULL_PARTITION), branch); + + // Concurrent Replace Partitions should fail with ValidationException + ReplacePartitions replace = table.newReplacePartitions(); + assertThatThrownBy( + () -> + commit( + table, + replace + .addFile(FILE_NULL_PARTITION) + .addFile(FILE_B) + .validateNoConflictingData() + .validateNoConflictingDeletes(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[data_bucket=null, data_bucket=1]: [/path/to/data-null-partition.parquet]"); + } + + @TestTemplate + public void testValidateWithVoidTransform() throws IOException { + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); + + Table tableVoid = TestTables.create(tableDir, "tablevoid", SCHEMA, SPEC_VOID, formatVersion); + commit(tableVoid, tableVoid.newReplacePartitions().addFile(FILE_A_VOID_PARTITION), branch); + + // Concurrent Replace Partitions should fail with ValidationException + ReplacePartitions replace = tableVoid.newReplacePartitions(); + assertThatThrownBy( + () -> + commit( + tableVoid, + replace + .addFile(FILE_A_VOID_PARTITION) + .addFile(FILE_B_VOID_PARTITION) + .validateNoConflictingData() + .validateNoConflictingDeletes(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[id_null=null, data_bucket=1, id_null=null, data_bucket=0]: " + + "[/path/to/data-a-void-partition.parquet]"); + } + @TestTemplate public void testConcurrentReplaceConflict() { commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 176f61079fc3..f1d23de32a42 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -48,7 +48,7 @@ public class TestRewriteManifests extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java index b326274842a3..10c22b759988 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -39,7 +39,7 @@ public class TestScanSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> 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<Object> 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<Object> 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<Object> 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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index 2ec6abd4e428..8a30036f3242 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -33,7 +33,7 @@ public class TestSnapshot extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java index 89312201265d..96dde0f48eb7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java @@ -43,7 +43,7 @@ public class TestSnapshotLoading extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> 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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 053a9c374178..529e0cc614f6 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -32,7 +32,7 @@ public class TestSnapshotSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index a3ba69a808b3..ad773192b417 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -76,7 +76,7 @@ public class TestSortOrder { @Parameters(name = "formatVersion = {0}") protected static List<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index e11cc500df55..5be8187a923e 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -32,6 +32,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.entry; +import static org.junit.jupiter.params.provider.Arguments.arguments; import com.fasterxml.jackson.core.JsonGenerator; import java.io.File; @@ -49,6 +50,8 @@ import java.util.Set; import java.util.SortedSet; import java.util.UUID; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.iceberg.TableMetadata.MetadataLogEntry; import org.apache.iceberg.TableMetadata.SnapshotLogEntry; import org.apache.iceberg.exceptions.ValidationException; @@ -61,8 +64,12 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; public class TestTableMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; @@ -1451,8 +1458,20 @@ public void testCreateV2MetadataThroughTableProperty() { .doesNotContainKey(TableProperties.FORMAT_VERSION); } - @Test - public void testReplaceV1MetadataToV2ThroughTableProperty() { + private static Stream<Arguments> upgradeFormatVersionProvider() { + // return a stream of all valid upgrade paths + return IntStream.range(1, TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION) + .boxed() + .flatMap( + baseFormatVersion -> + IntStream.rangeClosed( + baseFormatVersion + 1, TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION) + .mapToObj(newFormatVersion -> arguments(baseFormatVersion, newFormatVersion))); + } + + @ParameterizedTest + @MethodSource("upgradeFormatVersionProvider") + public void testReplaceMetadataThroughTableProperty(int baseFormatVersion, int newFormatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = @@ -1460,7 +1479,8 @@ public void testReplaceV1MetadataToV2ThroughTableProperty() { schema, PartitionSpec.unpartitioned(), null, - ImmutableMap.of(TableProperties.FORMAT_VERSION, "1", "key", "val")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(baseFormatVersion), "key", "val")); meta = meta.buildReplacement( @@ -1468,17 +1488,19 @@ public void testReplaceV1MetadataToV2ThroughTableProperty() { meta.spec(), meta.sortOrder(), meta.location(), - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key2", "val2")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(newFormatVersion), "key2", "val2")); - assertThat(meta.formatVersion()).isEqualTo(2); + assertThat(meta.formatVersion()).isEqualTo(newFormatVersion); assertThat(meta.properties()) .containsEntry("key", "val") .containsEntry("key2", "val2") .doesNotContainKey(TableProperties.FORMAT_VERSION); } - @Test - public void testUpgradeV1MetadataToV2ThroughTableProperty() { + @ParameterizedTest + @MethodSource("upgradeFormatVersionProvider") + public void testUpgradeMetadataThroughTableProperty(int baseFormatVersion, int newFormatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = @@ -1486,15 +1508,17 @@ public void testUpgradeV1MetadataToV2ThroughTableProperty() { schema, PartitionSpec.unpartitioned(), null, - ImmutableMap.of(TableProperties.FORMAT_VERSION, "1", "key", "val")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(baseFormatVersion), "key", "val")); meta = meta.replaceProperties( - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key2", "val2")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(newFormatVersion), "key2", "val2")); assertThat(meta.formatVersion()) .as("format version should be configured based on the format-version key") - .isEqualTo(2); + .isEqualTo(newFormatVersion); assertThat(meta.properties()) .as("should not contain format-version but should contain new properties") .containsExactly(entry("key2", "val2")); @@ -1612,4 +1636,54 @@ public void buildReplacementKeepsSnapshotLog() throws Exception { .hasSize(2) .containsExactlyElementsOf(metadata.snapshotLog()); } + + @Test + public void testConstructV3Metadata() { + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + 3); + } + + @Test + public void testV3TimestampNanoTypeSupport() { + Schema v3Schema = + new Schema( + Types.NestedField.required(3, "id", Types.LongType.get()), + Types.NestedField.required(4, "data", Types.StringType.get()), + Types.NestedField.required( + 5, + "struct", + Types.StructType.of( + Types.NestedField.optional( + 6, "ts_nanos", Types.TimestampNanoType.withZone())))); + + for (int unsupportedFormatVersion : ImmutableList.of(1, 2)) { + Assertions.assertThrows( + IllegalStateException.class, + () -> + TableMetadata.newTableMetadata( + v3Schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + unsupportedFormatVersion), + String.format( + "Invalid type in v%s schema: struct.ts_nanos timestamptz_ns is not supported until v3", + unsupportedFormatVersion)); + } + + // should be allowed in v3 + TableMetadata.newTableMetadata( + v3Schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + 3); + } } diff --git a/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<Object> 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<Object> 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/TestTables.java b/core/src/test/java/org/apache/iceberg/TestTables.java index de05e85c3c77..eeff5db8e5a6 100644 --- a/core/src/test/java/org/apache/iceberg/TestTables.java +++ b/core/src/test/java/org/apache/iceberg/TestTables.java @@ -216,6 +216,7 @@ public static class TestTableOperations implements TableOperations { private final String tableName; private final File metadata; + private final FileIO fileIO; private TableMetadata current = null; private long lastSnapshotId = 0; private int failCommits = 0; @@ -223,6 +224,22 @@ public static class TestTableOperations implements TableOperations { public TestTableOperations(String tableName, File location) { this.tableName = tableName; this.metadata = new File(location, "metadata"); + this.fileIO = new LocalFileIO(); + metadata.mkdirs(); + refresh(); + if (current != null) { + for (Snapshot snap : current.snapshots()) { + this.lastSnapshotId = Math.max(lastSnapshotId, snap.snapshotId()); + } + } else { + this.lastSnapshotId = 0; + } + } + + public TestTableOperations(String tableName, File location, FileIO fileIO) { + this.tableName = tableName; + this.metadata = new File(location, "metadata"); + this.fileIO = fileIO; metadata.mkdirs(); refresh(); if (current != null) { @@ -277,7 +294,7 @@ public void commit(TableMetadata base, TableMetadata updatedMetadata) { @Override public FileIO io() { - return new LocalFileIO(); + return fileIO; } @Override @@ -300,7 +317,7 @@ public long newSnapshotId() { } } - static class LocalFileIO implements FileIO { + public static class LocalFileIO implements FileIO { @Override public InputFile newInputFile(String path) { diff --git a/core/src/test/java/org/apache/iceberg/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<Object> 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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java index ed1142441736..1a6c289ea241 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java @@ -37,6 +37,8 @@ import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestUpdateRequirements { private final TableMetadata metadata = mock(TableMetadata.class); @@ -184,11 +186,12 @@ public void assignUUIDToViewFailure() { updatedViewMetadata.uuid(), viewMetadata.uuid())); } - @Test - public void upgradeFormatVersion() { + @ParameterizedTest + @ValueSource(ints = {2, 3}) + public void upgradeFormatVersion(int formatVersion) { List<UpdateRequirement> requirements = UpdateRequirements.forUpdateTable( - metadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(2))); + metadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(formatVersion))); requirements.forEach(req -> req.validate(metadata)); assertThat(requirements) 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<Object> 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<Object> parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java index 6512b2499071..2b342936fdd9 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java @@ -58,7 +58,7 @@ import org.junit.jupiter.params.provider.ValueSource; public class TestHadoopCatalog extends HadoopTableTestBase { - private static final ImmutableMap<String, String> meta = ImmutableMap.of(); + private static final ImmutableMap<String, String> META = ImmutableMap.of(); @ParameterizedTest @ValueSource(ints = {1, 2}) @@ -337,7 +337,7 @@ public void testCreateNamespace() throws Exception { TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "metadata"); TableIdentifier tbl2 = TableIdentifier.of("db", "ns2", "ns3", "tbl2"); - Lists.newArrayList(tbl1, tbl2).forEach(t -> catalog.createNamespace(t.namespace(), meta)); + Lists.newArrayList(tbl1, tbl2).forEach(t -> catalog.createNamespace(t.namespace(), META)); String metaLocation1 = warehouseLocation + "/" + "db/ns1/ns2"; FileSystem fs1 = Util.getFs(new Path(metaLocation1), catalog.getConf()); diff --git a/core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.java b/core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.java new file mode 100644 index 000000000000..e6225d886cee --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.io; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.times; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.TestTables; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; + +public class TestFileIOTracker { + + @TempDir private File tableDir; + + @SuppressWarnings("resource") + @Test + public void nullTableOps() { + assertThatThrownBy(() -> new FileIOTracker().track(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table ops: null"); + } + + @SuppressWarnings("unchecked") + @Test + public void fileIOGetsClosed() throws NoSuchFieldException, IllegalAccessException { + FileIOTracker fileIOTracker = new FileIOTracker(); + + FileIO firstFileIO = Mockito.spy(new TestTables.LocalFileIO()); + TestTables.TestTableOperations firstOps = + new TestTables.TestTableOperations("x", tableDir, firstFileIO); + fileIOTracker.track(firstOps); + assertThat(fileIOTracker.tracker().estimatedSize()).isEqualTo(1); + + FileIO secondFileIO = Mockito.spy(new TestTables.LocalFileIO()); + TestTables.TestTableOperations secondOps = + new TestTables.TestTableOperations("y", tableDir, secondFileIO); + fileIOTracker.track(secondOps); + assertThat(fileIOTracker.tracker().estimatedSize()).isEqualTo(2); + + fileIOTracker.close(); + Awaitility.await("FileIO gets closed") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted( + () -> { + assertThat(fileIOTracker.tracker().estimatedSize()).isEqualTo(0); + Mockito.verify(firstFileIO, times(1)).close(); + Mockito.verify(secondFileIO, times(1)).close(); + }); + } +} diff --git a/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<Object> 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/jdbc/TestJdbcCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java index 148a483b3477..d21605cace21 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -750,8 +750,11 @@ public void testListNamespace() { TableIdentifier tbl4 = TableIdentifier.of("db", "metadata"); TableIdentifier tbl5 = TableIdentifier.of("db2", "metadata"); TableIdentifier tbl6 = TableIdentifier.of("tbl6"); + TableIdentifier tbl7 = TableIdentifier.of("db2", "ns4", "tbl5"); + TableIdentifier tbl8 = TableIdentifier.of("d_", "ns5", "tbl6"); + TableIdentifier tbl9 = TableIdentifier.of("d%", "ns6", "tbl7"); - Lists.newArrayList(tbl1, tbl2, tbl3, tbl4, tbl5, tbl6) + Lists.newArrayList(tbl1, tbl2, tbl3, tbl4, tbl5, tbl6, tbl7, tbl8, tbl9) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); List<Namespace> nsp1 = catalog.listNamespaces(Namespace.of("db")); @@ -765,11 +768,19 @@ public void testListNamespace() { List<Namespace> nsp3 = catalog.listNamespaces(); Set<String> tblSet2 = Sets.newHashSet(nsp3.stream().map(Namespace::toString).iterator()); - assertThat(tblSet2).hasSize(3).contains("db", "db2", ""); + assertThat(tblSet2).hasSize(5).contains("db", "db2", "d_", "d%", ""); List<Namespace> nsp4 = catalog.listNamespaces(); Set<String> tblSet3 = Sets.newHashSet(nsp4.stream().map(Namespace::toString).iterator()); - assertThat(tblSet3).hasSize(3).contains("db", "db2", ""); + assertThat(tblSet3).hasSize(5).contains("db", "db2", "d_", "d%", ""); + + List<Namespace> nsp5 = catalog.listNamespaces(Namespace.of("d_")); + assertThat(nsp5).hasSize(1); + assertThat(nsp5.get(0)).hasToString("d_.ns5"); + + List<Namespace> nsp6 = catalog.listNamespaces(Namespace.of("d%")); + assertThat(nsp6).hasSize(1); + assertThat(nsp6.get(0)).hasToString("d%.ns6"); assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) 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<Object> 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 extends RESTResponse> 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/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index ced15b865115..1c15cfab43a3 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -155,15 +155,10 @@ public <T extends RESTResponse> T execute( } }; - RESTCatalogServlet servlet = new RESTCatalogServlet(adaptor); ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); - servletContext.setContextPath("/"); - ServletHolder servletHolder = new ServletHolder(servlet); - servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); - servletContext.addServlet(servletHolder, "/*"); - servletContext.setVirtualHosts(null); - servletContext.setGzipHandler(new GzipHandler()); + servletContext.addServlet(new ServletHolder(new RESTCatalogServlet(adaptor)), "/*"); + servletContext.setHandler(new GzipHandler()); this.httpServer = new Server(0); httpServer.setHandler(servletContext); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java index f67c4b078e5d..db0969620dc9 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -92,15 +92,11 @@ public <T extends RESTResponse> T execute( } }; - RESTCatalogServlet servlet = new RESTCatalogServlet(adaptor); ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); servletContext.setContextPath("/"); - ServletHolder servletHolder = new ServletHolder(servlet); - servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); - servletContext.addServlet(servletHolder, "/*"); - servletContext.setVirtualHosts(null); - servletContext.setGzipHandler(new GzipHandler()); + servletContext.addServlet(new ServletHolder(new RESTCatalogServlet(adaptor)), "/*"); + servletContext.setHandler(new GzipHandler()); this.httpServer = new Server(0); httpServer.setHandler(servletContext); diff --git a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java index af9c6ec5212c..5e37e0390db9 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -20,26 +20,29 @@ 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.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.HashMultiset; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMultiset; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Multiset; +import org.apache.iceberg.util.ParallelIterable.ParallelIterator; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; public class TestParallelIterable { @Test - public void closeParallelIteratorWithoutCompleteIteration() - throws IOException, IllegalAccessException, NoSuchFieldException { + public void closeParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); Iterable<CloseableIterable<Integer>> transform = @@ -57,25 +60,21 @@ public CloseableIterator<Integer> iterator() { }); ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(transform, executor); - CloseableIterator<Integer> iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue<?> queue = (ConcurrentLinkedQueue<?>) queueField.get(iterator); + ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) 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 - public void closeMoreDataParallelIteratorWithoutCompleteIteration() - throws IOException, IllegalAccessException, NoSuchFieldException { + public void closeMoreDataParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); Iterator<Integer> integerIterator = new Iterator<Integer>() { @@ -117,25 +116,98 @@ public CloseableIterator<Integer> iterator() { }); ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(transform, executor); - CloseableIterator<Integer> iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue<?> queue = (ConcurrentLinkedQueue<?>) queueField.get(iterator); + ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) 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)); } - private void queueHasElements(CloseableIterator<Integer> iterator, Queue queue) { + @Test + public void limitQueueSize() { + List<Iterable<Integer>> iterables = + ImmutableList.of( + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator()); + + Multiset<Integer> expectedValues = + IntStream.range(0, 100) + .boxed() + .flatMap(i -> Stream.of(i, i, i)) + .collect(ImmutableMultiset.toImmutableMultiset()); + + int maxQueueSize = 20; + ExecutorService executor = Executors.newCachedThreadPool(); + ParallelIterable<Integer> parallelIterable = + new ParallelIterable<>(iterables, executor, maxQueueSize); + ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator(); + + Multiset<Integer> actualValues = HashMultiset.create(); + + while (iterator.hasNext()) { + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(maxQueueSize + iterables.size()); + actualValues.add(iterator.next()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + executor.shutdownNow(); + } + + @Test + public void queueSizeOne() { + List<Iterable<Integer>> iterables = + ImmutableList.of( + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator(), + () -> IntStream.range(0, 100).iterator()); + + Multiset<Integer> expectedValues = + IntStream.range(0, 100) + .boxed() + .flatMap(i -> Stream.of(i, i, i)) + .collect(ImmutableMultiset.toImmutableMultiset()); + + ExecutorService executor = Executors.newCachedThreadPool(); + ParallelIterable<Integer> parallelIterable = new ParallelIterable<>(iterables, executor, 1); + ParallelIterator<Integer> iterator = (ParallelIterator<Integer>) parallelIterable.iterator(); + + Multiset<Integer> actualValues = HashMultiset.create(); + + while (iterator.hasNext()) { + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(1 + iterables.size()); + actualValues.add(iterator.next()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + executor.shutdownNow(); + } + + private void queueHasElements(ParallelIterator<Integer> 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/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java index 574acf15cb9b..2f4a55b6fdc3 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java @@ -97,7 +97,7 @@ public static List<Object> parameters() { return Arrays.asList(FileFormat.PARQUET, FileFormat.ORC); } - private static final Types.StructType structFieldType = + private static final Types.StructType STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(8, "int_field", IntegerType.get())); private static final Schema SCHEMA = @@ -108,7 +108,7 @@ public static List<Object> parameters() { optional(4, "all_nulls", DoubleType.get()), optional(5, "some_nulls", StringType.get()), optional(6, "no_nulls", StringType.get()), - optional(7, "struct_not_null", structFieldType), + optional(7, "struct_not_null", STRUCT_FIELD_TYPE), optional(9, "not_in_file", FloatType.get()), optional(10, "str", StringType.get()), optional( @@ -120,7 +120,7 @@ public static List<Object> parameters() { optional(16, "no_nans", DoubleType.get()), optional(17, "some_double_nans", DoubleType.get())); - private static final Types.StructType _structFieldType = + private static final Types.StructType UNDERSCORE_STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(8, "_int_field", IntegerType.get())); private static final Schema FILE_SCHEMA = @@ -131,7 +131,7 @@ public static List<Object> parameters() { optional(4, "_all_nulls", DoubleType.get()), optional(5, "_some_nulls", StringType.get()), optional(6, "_no_nulls", StringType.get()), - optional(7, "_struct_not_null", _structFieldType), + optional(7, "_struct_not_null", UNDERSCORE_STRUCT_FIELD_TYPE), optional(10, "_str", StringType.get()), optional(14, "_all_nans", Types.DoubleType.get()), optional(15, "_some_nans", FloatType.get()), @@ -202,7 +202,7 @@ public void createOrcInputFile() throws IOException { "_some_double_nans", (i % 10 == 0) ? Double.NaN : 2D); // includes some nan values record.setField("_no_nans", 3D); // optional, but always non-nan - GenericRecord structNotNull = GenericRecord.create(_structFieldType); + GenericRecord structNotNull = GenericRecord.create(UNDERSCORE_STRUCT_FIELD_TYPE); structNotNull.setField("_int_field", INT_MIN_VALUE + i); record.setField("_struct_not_null", structNotNull); // struct with int @@ -225,7 +225,7 @@ private void createParquetInputFile() throws IOException { assertThat(parquetFile.delete()).isTrue(); // build struct field schema - org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); + org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); OutputFile outFile = Files.localOutput(parquetFile); try (FileAppender<Record> appender = Parquet.write(outFile).schema(FILE_SCHEMA).build()) { diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java index 75b19554ef2a..19ae28a44045 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java @@ -126,16 +126,16 @@ public class TestMetricsRowGroupFilterTypes { private static MessageType parquetSchema = null; private static BlockMetaData rowGroupMetadata = null; - private static final UUID uuid = UUID.randomUUID(); - private static final LocalDate date = + private static final UUID UUID_VALUE = UUID.randomUUID(); + private static final LocalDate DATE = LocalDate.parse("2018-06-29", DateTimeFormatter.ISO_LOCAL_DATE); - private static final LocalTime time = + private static final LocalTime TIME = LocalTime.parse("10:02:34.000000", DateTimeFormatter.ISO_LOCAL_TIME); - private static final OffsetDateTime timestamptz = + private static final OffsetDateTime TIMESTAMPTZ = OffsetDateTime.parse("2018-06-29T10:02:34.000000+00:00", DateTimeFormatter.ISO_DATE_TIME); - private static final LocalDateTime timestamp = + private static final LocalDateTime TIMESTAMP = LocalDateTime.parse("2018-06-29T10:02:34.000000", DateTimeFormatter.ISO_LOCAL_DATE_TIME); - private static final byte[] fixed = "abcd".getBytes(StandardCharsets.UTF_8); + private static final byte[] FIXED = "abcd".getBytes(StandardCharsets.UTF_8); @BeforeEach public void createInputFile() throws IOException { @@ -148,15 +148,16 @@ public void createInputFile() throws IOException { record.setField("_long", 5_000_000_000L + i); record.setField("_float", ((float) (100 - i)) / 100F + 1.0F); // 2.0f, 1.99f, 1.98f, ... record.setField("_double", ((double) i) / 100.0D + 2.0D); // 2.0d, 2.01d, 2.02d, ... - record.setField("_date", date); - record.setField("_time", time); - record.setField("_timestamp", timestamp); - record.setField("_timestamptz", timestamptz); + record.setField("_date", DATE); + record.setField("_time", TIME); + record.setField("_timestamp", TIMESTAMP); + record.setField("_timestamptz", TIMESTAMPTZ); record.setField("_string", "tapir"); - // record.setField("_uuid", uuid); // Disable writing UUID value as GenericParquetWriter does + // record.setField("_uuid", UUID_VALUE); // Disable writing UUID value as GenericParquetWriter + // does // not handle UUID type // correctly; Also UUID tests are disabled for both ORC and Parquet anyway - record.setField("_fixed", fixed); + record.setField("_fixed", FIXED); record.setField("_binary", ByteBuffer.wrap("xyz".getBytes(StandardCharsets.UTF_8))); record.setField("_int_decimal", new BigDecimal("77.77")); record.setField("_long_decimal", new BigDecimal("88.88")); @@ -254,7 +255,7 @@ public static Object[][] parameters() { "2018-06-29T10:02:34.000000-07:00" }, {FileFormat.PARQUET, "string", "tapir", "monthly"}, - // { FileFormat.PARQUET, "uuid", uuid, UUID.randomUUID() }, // not supported yet + // { FileFormat.PARQUET, "uuid", UUID_VALUE, UUID.randomUUID() }, // not supported yet { FileFormat.PARQUET, "fixed", @@ -286,7 +287,7 @@ public static Object[][] parameters() { }, {FileFormat.ORC, "string", "tapir", "monthly"}, // uuid, fixed and binary types not supported yet - // { FileFormat.ORC, "uuid", uuid, UUID.randomUUID() }, + // { FileFormat.ORC, "uuid", UUID_VALUE, UUID.randomUUID() }, // { FileFormat.ORC, "fixed", "abcd".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, // 2, 3 } }, // { FileFormat.ORC, "binary", "xyz".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, diff --git a/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java index c6a5ed9f6d86..4b0a10830221 100644 --- a/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java @@ -53,8 +53,8 @@ import org.junit.jupiter.api.Test; public class TestParquetEncryptionWithWriteSupport extends DataTest { - private static final ByteBuffer fileDek = ByteBuffer.allocate(16); - private static final ByteBuffer aadPrefix = ByteBuffer.allocate(16); + private static final ByteBuffer FILE_DEK = ByteBuffer.allocate(16); + private static final ByteBuffer AAD_PREFIX = ByteBuffer.allocate(16); @Override protected void writeAndValidate(Schema schema) throws IOException { @@ -64,14 +64,14 @@ protected void writeAndValidate(Schema schema) throws IOException { assertThat(testFile.delete()).isTrue(); SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); try (FileAppender<Record> appender = Parquet.write(Files.localOutput(testFile)) .schema(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .createWriterFunc(GenericParquetWriter::buildWriter) .build()) { appender.addAll(expected); @@ -92,8 +92,8 @@ protected void writeAndValidate(Schema schema) throws IOException { try (CloseableIterable<Record> reader = Parquet.read(Files.localInput(testFile)) .project(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { rows = Lists.newArrayList(reader); @@ -107,8 +107,8 @@ protected void writeAndValidate(Schema schema) throws IOException { try (CloseableIterable<Record> reader = Parquet.read(Files.localInput(testFile)) .project(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .reuseContainers() .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { @@ -134,10 +134,12 @@ public void testTwoLevelList() throws IOException { assertThat(testFile.delete()).isTrue(); SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); FileEncryptionProperties fileEncryptionProperties = - FileEncryptionProperties.builder(fileDek.array()).withAADPrefix(aadPrefix.array()).build(); + FileEncryptionProperties.builder(FILE_DEK.array()) + .withAADPrefix(AAD_PREFIX.array()) + .build(); ParquetWriter<org.apache.avro.generic.GenericRecord> writer = AvroParquetWriter.<org.apache.avro.generic.GenericRecord>builder(new Path(testFile.toURI())) @@ -164,8 +166,8 @@ public void testTwoLevelList() throws IOException { try (CloseableIterable<Record> reader = Parquet.read(Files.localInput(testFile)) .project(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .reuseContainers() .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { diff --git a/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java b/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java index 74105458dbcc..e4f6c028bc8e 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java +++ b/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java @@ -73,10 +73,10 @@ public abstract class TestWriterMetrics<T> { // create a schema with all supported fields protected static final Schema SCHEMA = new Schema(ID_FIELD, DATA_FIELD, STRUCT_FIELD); - protected static final SortOrder sortOrder = + protected static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").asc("structField.longValue").build(); - protected static final Map<String, String> properties = + protected static final Map<String, String> PROPERTIES = ImmutableMap.of(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); @TempDir private File tempDir; @@ -107,7 +107,7 @@ public void setupTable() throws Exception { this.table = TestTables.create( - tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), sortOrder, FORMAT_V2); + tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SORT_ORDER, FORMAT_V2); table.updateProperties().set(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none").commit(); this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); diff --git a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java index 945e89670de8..01a998c65efe 100644 --- a/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java +++ b/delta-lake/src/integration/java/org/apache/iceberg/delta/TestSnapshotDeltaLakeTable.java @@ -70,9 +70,9 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String DELTA_SOURCE_VALUE = "delta"; private static final String ORIGINAL_LOCATION_PROP = "original_location"; private static final String NAMESPACE = "delta_conversion_test"; - private static final String defaultSparkCatalog = "spark_catalog"; - private static final String icebergCatalogName = "iceberg_hive"; - private static final Map<String, String> config = + private static final String DEFAULT_SPARK_CATALOG = "spark_catalog"; + private static final String ICEBERG_CATALOG_NAME = "iceberg_hive"; + private static final Map<String, String> CONFIG = ImmutableMap.of( "type", "hive", "default-namespace", "default", @@ -87,8 +87,8 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { @TempDir private File tempB; public TestSnapshotDeltaLakeTable() { - super(icebergCatalogName, SparkCatalog.class.getName(), config); - spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); + super(ICEBERG_CATALOG_NAME, SparkCatalog.class.getName(), CONFIG); + spark.conf().set("spark.sql.catalog." + DEFAULT_SPARK_CATALOG, DeltaCatalog.class.getName()); } @BeforeAll @@ -152,14 +152,14 @@ public static void afterClass() { @Test public void testBasicSnapshotPartitioned() { - String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); + String partitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "partitioned_table"); String partitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_partitioned_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_partitioned_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) @@ -173,14 +173,14 @@ public void testBasicSnapshotPartitioned() { @Test public void testBasicSnapshotUnpartitioned() { - String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); + String unpartitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "unpartitioned_table"); String unpartitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_unpartitioned_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_unpartitioned_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) @@ -194,7 +194,7 @@ public void testBasicSnapshotUnpartitioned() { @Test public void testSnapshotWithNewLocation() { - String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); + String partitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "partitioned_table"); String partitionedLocation = tempA.toURI().toString(); String newIcebergTableLocation = tempB.toURI().toString(); @@ -202,7 +202,7 @@ public void testSnapshotWithNewLocation() { spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_new_table_location_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_new_table_location_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) @@ -217,7 +217,7 @@ public void testSnapshotWithNewLocation() { @Test public void testSnapshotWithAdditionalProperties() { - String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); + String unpartitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "unpartitioned_table"); String unpartitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); @@ -230,7 +230,8 @@ public void testSnapshotWithAdditionalProperties() { + unpartitionedIdentifier + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_additional_properties_table"); + String newTableIdentifier = + destName(ICEBERG_CATALOG_NAME, "iceberg_additional_properties_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) @@ -255,8 +256,9 @@ public void testSnapshotWithAdditionalProperties() { @Test public void testSnapshotTableWithExternalDataFiles() { - String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String externalDataFilesIdentifier = destName(defaultSparkCatalog, "external_data_files_table"); + String unpartitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "unpartitioned_table"); + String externalDataFilesIdentifier = + destName(DEFAULT_SPARK_CATALOG, "external_data_files_table"); String unpartitionedLocation = tempA.toURI().toString(); String externalDataFilesTableLocation = tempB.toURI().toString(); @@ -269,7 +271,7 @@ public void testSnapshotTableWithExternalDataFiles() { // are not at the same location as the table. addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_external_data_files_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_external_data_files_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, externalDataFilesTableLocation) @@ -283,7 +285,7 @@ public void testSnapshotTableWithExternalDataFiles() { @Test public void testSnapshotSupportedTypes() { - String typeTestIdentifier = destName(defaultSparkCatalog, "type_test_table"); + String typeTestIdentifier = destName(DEFAULT_SPARK_CATALOG, "type_test_table"); String typeTestTableLocation = tempA.toURI().toString(); writeDeltaTable( @@ -294,7 +296,7 @@ public void testSnapshotSupportedTypes() { "timestampStrCol", "booleanCol", "longCol"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_type_test_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_type_test_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, typeTestTableLocation) @@ -308,7 +310,7 @@ public void testSnapshotSupportedTypes() { @Test public void testSnapshotVacuumTable() throws IOException { - String vacuumTestIdentifier = destName(defaultSparkCatalog, "vacuum_test_table"); + String vacuumTestIdentifier = destName(DEFAULT_SPARK_CATALOG, "vacuum_test_table"); String vacuumTestTableLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation); @@ -330,7 +332,7 @@ public void testSnapshotVacuumTable() throws IOException { assertThat(deleteResult).isTrue(); spark.sql("VACUUM " + vacuumTestIdentifier + " RETAIN 0 HOURS"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_vacuum_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_vacuum_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, vacuumTestTableLocation) @@ -343,7 +345,7 @@ public void testSnapshotVacuumTable() throws IOException { @Test public void testSnapshotLogCleanTable() throws IOException { - String logCleanTestIdentifier = destName(defaultSparkCatalog, "log_clean_test_table"); + String logCleanTestIdentifier = destName(DEFAULT_SPARK_CATALOG, "log_clean_test_table"); String logCleanTestTableLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, logCleanTestIdentifier, logCleanTestTableLocation, "id"); @@ -364,7 +366,7 @@ public void testSnapshotLogCleanTable() throws IOException { logCleanTestTableLocation.concat("/_delta_log/00000000000000000000.json")))); assertThat(deleteResult).isTrue(); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_log_clean_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_log_clean_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, logCleanTestTableLocation) @@ -477,7 +479,7 @@ private Table getIcebergTable(String icebergTableIdentifier) { } private String destName(String catalogName, String dest) { - if (catalogName.equals(defaultSparkCatalog)) { + if (catalogName.equals(DEFAULT_SPARK_CATALOG)) { return NAMESPACE + "." + catalogName + "_" + dest; } return catalogName + "." + NAMESPACE + "." + catalogName + "_" + dest; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java index 8699eb3b5d26..f43ce39d1c5f 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java @@ -46,13 +46,13 @@ static DeltaLakeToIcebergMigrationActionsProvider defaultActions() { class DefaultDeltaLakeToIcebergMigrationActions implements DeltaLakeToIcebergMigrationActionsProvider { - private static final DefaultDeltaLakeToIcebergMigrationActions defaultMigrationActions = + private static final DefaultDeltaLakeToIcebergMigrationActions DEFAULT_MIGRATION_ACTIONS = new DefaultDeltaLakeToIcebergMigrationActions(); private DefaultDeltaLakeToIcebergMigrationActions() {} static DefaultDeltaLakeToIcebergMigrationActions defaultMigrationActions() { - return defaultMigrationActions; + return DEFAULT_MIGRATION_ACTIONS; } } } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index 6d99d64470f8..817310203109 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -38,13 +38,13 @@ import org.junit.jupiter.api.Test; public class TestDeltaLakeTypeToType { - private static final String optionalBooleanType = "testNullableBoolType"; - private static final String requiredBinaryType = "testRequiredBinaryType"; - private static final String doubleArrayType = "testNullableArrayType"; - private static final String structArrayType = "testStructArrayType"; - private static final String innerAtomicSchema = "testInnerAtomicSchema"; - private static final String stringLongMapType = "testStringLongMap"; - private static final String nullType = "testNullType"; + private static final String OPTIONAL_BOOLEAN_TYPE = "testNullableBoolType"; + private static final String REQUIRED_BINARY_TYPE = "testRequiredBinaryType"; + private static final String DOUBLE_ARRAY_TYPE = "testNullableArrayType"; + private static final String STRUCT_ARRAY_TYPE = "testStructArrayType"; + private static final String INNER_ATOMIC_SCHEMA = "testInnerAtomicSchema"; + private static final String STRING_LONG_MAP_TYPE = "testStringLongMap"; + private static final String NULL_TYPE = "testNullType"; private StructType deltaAtomicSchema; private StructType deltaNestedSchema; private StructType deltaShallowNullTypeSchema; @@ -54,20 +54,20 @@ public class TestDeltaLakeTypeToType { public void constructDeltaLakeSchema() { deltaAtomicSchema = new StructType() - .add(optionalBooleanType, new BooleanType()) - .add(requiredBinaryType, new BinaryType(), false); + .add(OPTIONAL_BOOLEAN_TYPE, new BooleanType()) + .add(REQUIRED_BINARY_TYPE, new BinaryType(), false); deltaNestedSchema = new StructType() - .add(innerAtomicSchema, deltaAtomicSchema) - .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) - .add(structArrayType, new ArrayType(deltaAtomicSchema, true), false) - .add(stringLongMapType, new MapType(new StringType(), new LongType(), false), false); + .add(INNER_ATOMIC_SCHEMA, deltaAtomicSchema) + .add(DOUBLE_ARRAY_TYPE, new ArrayType(new DoubleType(), true), false) + .add(STRUCT_ARRAY_TYPE, new ArrayType(deltaAtomicSchema, true), false) + .add(STRING_LONG_MAP_TYPE, new MapType(new StringType(), new LongType(), false), false); deltaNullTypeSchema = new StructType() - .add(innerAtomicSchema, deltaAtomicSchema) - .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) - .add(stringLongMapType, new MapType(new NullType(), new LongType(), false), false); - deltaShallowNullTypeSchema = new StructType().add(nullType, new NullType(), false); + .add(INNER_ATOMIC_SCHEMA, deltaAtomicSchema) + .add(DOUBLE_ARRAY_TYPE, new ArrayType(new DoubleType(), true), false) + .add(STRING_LONG_MAP_TYPE, new MapType(new NullType(), new LongType(), false), false); + deltaShallowNullTypeSchema = new StructType().add(NULL_TYPE, new NullType(), false); } @Test @@ -77,10 +77,11 @@ public void testAtomicTypeConversion() { deltaAtomicSchema, new DeltaLakeTypeToType(deltaAtomicSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - assertThat(convertedSchema.findType(optionalBooleanType)).isInstanceOf(Types.BooleanType.class); - assertThat(convertedSchema.findField(optionalBooleanType).isOptional()).isTrue(); - assertThat(convertedSchema.findType(requiredBinaryType)).isInstanceOf(Types.BinaryType.class); - assertThat(convertedSchema.findField(requiredBinaryType).isRequired()).isTrue(); + assertThat(convertedSchema.findType(OPTIONAL_BOOLEAN_TYPE)) + .isInstanceOf(Types.BooleanType.class); + assertThat(convertedSchema.findField(OPTIONAL_BOOLEAN_TYPE).isOptional()).isTrue(); + assertThat(convertedSchema.findType(REQUIRED_BINARY_TYPE)).isInstanceOf(Types.BinaryType.class); + assertThat(convertedSchema.findField(REQUIRED_BINARY_TYPE).isRequired()).isTrue(); } @Test @@ -90,72 +91,74 @@ public void testNestedTypeConversion() { deltaNestedSchema, new DeltaLakeTypeToType(deltaNestedSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - assertThat(convertedSchema.findType(innerAtomicSchema)).isInstanceOf(Types.StructType.class); - assertThat(convertedSchema.findField(innerAtomicSchema).isOptional()).isTrue(); + assertThat(convertedSchema.findType(INNER_ATOMIC_SCHEMA)).isInstanceOf(Types.StructType.class); + assertThat(convertedSchema.findField(INNER_ATOMIC_SCHEMA).isOptional()).isTrue(); assertThat( convertedSchema - .findType(innerAtomicSchema) + .findType(INNER_ATOMIC_SCHEMA) .asStructType() - .fieldType(optionalBooleanType)) + .fieldType(OPTIONAL_BOOLEAN_TYPE)) .isInstanceOf(Types.BooleanType.class); assertThat( convertedSchema - .findType(innerAtomicSchema) + .findType(INNER_ATOMIC_SCHEMA) .asStructType() - .fieldType(requiredBinaryType)) + .fieldType(REQUIRED_BINARY_TYPE)) .isInstanceOf(Types.BinaryType.class); assertThat( convertedSchema - .findType(innerAtomicSchema) + .findType(INNER_ATOMIC_SCHEMA) .asStructType() - .field(requiredBinaryType) + .field(REQUIRED_BINARY_TYPE) .isRequired()) .isTrue(); - assertThat(convertedSchema.findType(stringLongMapType)).isInstanceOf(Types.MapType.class); - assertThat(convertedSchema.findType(stringLongMapType).asMapType().keyType()) + assertThat(convertedSchema.findType(STRING_LONG_MAP_TYPE)).isInstanceOf(Types.MapType.class); + assertThat(convertedSchema.findType(STRING_LONG_MAP_TYPE).asMapType().keyType()) .isInstanceOf(Types.StringType.class); - assertThat(convertedSchema.findType(stringLongMapType).asMapType().valueType()) + assertThat(convertedSchema.findType(STRING_LONG_MAP_TYPE).asMapType().valueType()) .isInstanceOf(Types.LongType.class); - assertThat(convertedSchema.findType(doubleArrayType)).isInstanceOf(Types.ListType.class); - assertThat(convertedSchema.findField(doubleArrayType).isRequired()).isTrue(); - assertThat(convertedSchema.findType(doubleArrayType).asListType().isElementOptional()).isTrue(); - assertThat(convertedSchema.findType(structArrayType)).isInstanceOf(Types.ListType.class); - assertThat(convertedSchema.findField(structArrayType).isRequired()).isTrue(); - assertThat(convertedSchema.findType(structArrayType).asListType().isElementOptional()).isTrue(); - assertThat(convertedSchema.findType(structArrayType).asListType().elementType()) + assertThat(convertedSchema.findType(DOUBLE_ARRAY_TYPE)).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField(DOUBLE_ARRAY_TYPE).isRequired()).isTrue(); + assertThat(convertedSchema.findType(DOUBLE_ARRAY_TYPE).asListType().isElementOptional()) + .isTrue(); + assertThat(convertedSchema.findType(STRUCT_ARRAY_TYPE)).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField(STRUCT_ARRAY_TYPE).isRequired()).isTrue(); + assertThat(convertedSchema.findType(STRUCT_ARRAY_TYPE).asListType().isElementOptional()) + .isTrue(); + assertThat(convertedSchema.findType(STRUCT_ARRAY_TYPE).asListType().elementType()) .isInstanceOf(Types.StructType.class); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .fieldType(optionalBooleanType)) + .fieldType(OPTIONAL_BOOLEAN_TYPE)) .isInstanceOf(Types.BooleanType.class); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .field(optionalBooleanType) + .field(OPTIONAL_BOOLEAN_TYPE) .isOptional()) .isTrue(); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .fieldType(requiredBinaryType)) + .fieldType(REQUIRED_BINARY_TYPE)) .isInstanceOf(Types.BinaryType.class); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .field(requiredBinaryType) + .field(REQUIRED_BINARY_TYPE) .isRequired()) .isTrue(); } diff --git a/deploy.gradle b/deploy.gradle index 3e2eda2a5a60..37a6262f1efe 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -17,17 +17,13 @@ * 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 { - if (it.name == 'iceberg-open-api') { - // don't publish iceberg-open-api - return - } - def isBom = it.name == 'iceberg-bom' + def isOpenApi = it.name == 'iceberg-open-api' apply plugin: 'maven-publish' apply plugin: 'signing' @@ -76,6 +72,9 @@ subprojects { apache(MavenPublication) { if (isBom) { from components.javaPlatform + } else if (isOpenApi) { + artifact testJar + artifact testFixturesJar } else { if (tasks.matching({task -> task.name == 'shadowJar'}).isEmpty()) { from components.java diff --git a/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/doap.rdf b/doap.rdf index 7f49fc42f793..51896c6a8037 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ <category rdf:resource="https://projects.apache.org/category/data-engineering" /> <release> <Version> - <name>1.5.2</name> - <created>2024-05-09</created> - <revision>1.5.2</revision> + <name>1.6.1</name> + <created>2024-08-27</created> + <revision>1.6.1</revision> </Version> </release> <repository> diff --git a/docs/docs/aws.md b/docs/docs/aws.md index bba968fa5586..973248a2b87c 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -468,13 +468,13 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata --conf spark.sql.catalog.my_catalog.type=glue \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.use-arn-region-enabled=false \ - --conf spark.sql.catalog.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::<ACCOUNT_ID>:accesspoint/<MRAP_ALIAS> \ + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3::<ACCOUNT_ID>:accesspoint/<MRAP_ALIAS> ``` -For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap` +For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3::<ACCOUNT_ID>:accesspoint/<MRAP_ALIAS>` access-point for all S3 operations. -For more details on using access-points, please refer [Using access points with compatible Amazon S3 operations](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html). +For more details on using access-points, please refer [Using access points with compatible Amazon S3 operations](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html), [Sample notebook](https://github.com/aws-samples/quant-research/tree/main) . ### S3 Access Grants 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/configuration.md b/docs/docs/configuration.md index 117adca09f68..264b9edfa7cc 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -67,7 +67,7 @@ Iceberg tables support table properties to configure table behavior, like the de | write.metadata.metrics.column.col1 | (not set) | Metrics mode for column 'col1' to allow per-column tuning; none, counts, truncate(length), or full | | write.target-file-size-bytes | 536870912 (512 MB) | Controls the size of files generated to target about this many bytes | | write.delete.target-file-size-bytes | 67108864 (64 MB) | Controls the size of delete files generated to target about this many bytes | -| write.distribution-mode | none | Defines distribution of write data: __none__: don't shuffle rows; __hash__: hash distribute by partition key ; __range__: range distribute by partition key or sort key if table has an SortOrder | +| write.distribution-mode | none, see engines for specific defaults, for example [Spark Writes](spark-writes.md#writing-distribution-modes) | Defines distribution of write data: __none__: don't shuffle rows; __hash__: hash distribute by partition key ; __range__: range distribute by partition key or sort key if table has an SortOrder | | write.delete.distribution-mode | hash | Defines distribution of write delete data | | write.update.distribution-mode | hash | Defines distribution of write update data | | write.merge.distribution-mode | none | Defines distribution of write merge data | diff --git a/docs/docs/flink-configuration.md b/docs/docs/flink-configuration.md index 42dc15f5b3d2..1ac16d7fc3e9 100644 --- a/docs/docs/flink-configuration.md +++ b/docs/docs/flink-configuration.md @@ -146,14 +146,56 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ ... ``` -| Flink option | Default | Description | -| ---------------------- | ------------------------------------------ | ------------------------------------------------------------ | -| write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | -| target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | -| upsert-enabled | Table write.upsert.enabled | Overrides this table's write.upsert.enabled | -| overwrite-enabled | false | Overwrite the table's data, overwrite mode shouldn't be enable when configuring to use UPSERT data stream. | -| distribution-mode | Table write.distribution-mode | Overrides this table's write.distribution-mode | -| compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | -| compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | -| compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | -| write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | +| Flink option | Default | Description | +|-----------------------------------------|--------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------| +| write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | +| target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | +| upsert-enabled | Table write.upsert.enabled | Overrides this table's write.upsert.enabled | +| overwrite-enabled | false | Overwrite the table's data, overwrite mode shouldn't be enable when configuring to use UPSERT data stream. | +| distribution-mode | Table write.distribution-mode | Overrides this table's write.distribution-mode. RANGE distribution is in experimental status. | +| range-distribution-statistics-type | Auto | Range distribution data statistics collection type: Map, Sketch, Auto. See details [here](#range-distribution-statistics-type). | +| range-distribution-sort-key-base-weight | 0.0 (double) | Base weight for every sort key relative to target traffic weight per writer task. See details [here](#range-distribution-sort-key-base-weight). | +| compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | +| compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | +| compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | +| write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | + +#### Range distribution statistics type + +Config value is a enum type: `Map`, `Sketch`, `Auto`. +<ul> +<li>Map: collects accurate sampling count for every single key. +It should be used for low cardinality scenarios (like hundreds or thousands). +<li>Sketch: constructs a uniform random sampling via reservoir sampling. +It fits well for high cardinality scenarios (like millions), as memory footprint is kept low. +<li>Auto: starts with Maps statistics. But if cardinality is detected higher +than a threshold (currently 10,000), statistics are automatically switched to Sketch. +</ul> + +#### Range distribution sort key base weight + +`range-distribution-sort-key-base-weight`: `0.0`. + +If sort order contains partition columns, each sort key would map to one partition and data +file. This relative weight can avoid placing too many small files for sort keys with low +traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means +each key has a base weight of `2%` of the targeted traffic weight per writer task. + +E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream +contains events from now up to 180 days ago. With event time, traffic weight distribution +across different days typically has a long tail pattern. Current day contains the most +traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism +is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer +task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, +the range partitioner would put all the oldest 150 days in one writer task. That writer task +would write to 150 small files (one per day). Keeping 150 open files can potentially consume +large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time +can also be potentially slow. If this config is set to `0.02`. It means every sort key has a +base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially +avoid placing more than `50` data files (one per day) on one writer task no matter how small +they are. + +This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For +{@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as +partition columns. Otherwise, too many partitions and small files may be generated during +write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. \ No newline at end of file diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index b916a5f9b7b0..f53b5d832efe 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -262,6 +262,107 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ Check out all the options here: [write-options](flink-configuration.md#write-options) +## Distribution mode + +Flink streaming writer supports both `HASH` and `RANGE` distribution mode. +You can enable it via `FlinkSink#Builder#distributionMode(DistributionMode)` +or via [write-options](flink-configuration.md#write-options). + +### Hash distribution + +HASH distribution shuffles data by partition key (partitioned table) or +equality fields (non-partitioned table). It simply leverages Flink's +`DataStream#keyBy` to distribute the data. + +HASH distribution has a few limitations. +<ul> +<li>It doesn't handle skewed data well. E.g. some partitions have a lot more data than others. +<li>It can result in unbalanced traffic distribution if cardinality of the partition key or +equality fields is low as demonstrated by [PR 4228](https://github.com/apache/iceberg/pull/4228). +<li>Writer parallelism is limited to the cardinality of the hash key. +If the cardinality is 10, only at most 10 writer tasks would get the traffic. +Having higher writer parallelism (even if traffic volume requires) won't help. +</ul> + +### Range distribution (experimental) + +RANGE distribution shuffles data by partition key or sort order via a custom range partitioner. +Range distribution collects traffic statistics to guide the range partitioner to +evenly distribute traffic to writer tasks. + +Range distribution only shuffle the data via range partitioner. Rows are *not* sorted within +a data file, which Flink streaming writer doesn't support yet. + +#### Use cases + +RANGE distribution can be applied to an Iceberg table that either is partitioned or +has SortOrder defined. For a partitioned table without SortOrder, partition columns +are used as sort order. If SortOrder is explicitly defined for the table, it is used by +the range partitioner. + +Range distribution can handle skewed data. E.g. +<ul> +<li>Table is partitioned by event time. Typically, recent hours have more data, +while the long-tail hours have less and less data. +<li>Table is partitioned by country code, where some countries (like US) have +a lot more traffic and smaller countries have a lot less data +<li>Table is partitioned by event type, where some types have a lot more data than others. +</ul> + +Range distribution can also cluster data on non-partition columns. +E.g., table is partitioned hourly on ingestion time. Queries often include +predicate on a non-partition column like `device_id` or `country_code`. +Range partition would improve the query performance by clustering on the non-partition column +when table `SortOrder` is defined with the non-partition column. + +#### Traffic statistics + +Statistics are collected by every shuffle operator subtask and aggregated by the coordinator +for every checkpoint cycle. Aggregated statistics are broadcast to all subtasks and +applied to the range partitioner in the next checkpoint. So it may take up to two checkpoint +cycles to detect traffic distribution change and apply the new statistics to range partitioner. + +Range distribution can work with low cardinality (like `country_code`) +or high cardinality (like `device_id`) scenarios. +<ul> +<li>For low cardinality scenario (like hundreds or thousands), +HashMap is used to track traffic distribution for every key. +If a new sort key value shows up, range partitioner would just +round-robin it to the writer tasks before traffic distribution has been learned +about the new key. +<li>For high cardinality scenario (like millions or billions), +uniform random sampling (reservoir sampling) is used to compute range bounds +that split the sort key space evenly. +It keeps the memory footprint and network exchange low. +Reservoir sampling work well if key distribution is relatively even. +If a single hot key has unbalanced large share of the traffic, +range split by uniform sampling probably won't work very well. +</ul> + +#### Usage + +Here is how to enable range distribution in Java. There are two optional advanced configs. Default should +work well for most cases. See [write-options](flink-configuration.md#write-options) for details. +```java +FlinkSink.forRowData(input) + ... + .distributionMode(DistributionMode.RANGE) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .rangeDistributionSortKeyBaseWeight(0.0d) + .append(); +``` + +### Overhead + +Data shuffling (hash or range) has computational overhead of serialization/deserialization +and network I/O. Expect some increase of CPU utilization. + +Range distribution also collect and aggregate data distribution statistics. +That would also incur some CPU overhead. Memory overhead is typically +small if using default statistics type of `Auto`. Don't use `Map` statistics +type if key cardinality is high. That could result in significant memory footprint +and large network exchange for statistics aggregation. + ## Notes Flink streaming write jobs rely on snapshot summary to keep the last committed checkpoint ID, and diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 9ff739649839..5b281b19891a 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -190,6 +190,7 @@ df.write | compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | | compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | | compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | +| distribution-mode | See [Spark Writes](spark-writes.md#writing-distribution-modes) for defaults | Override this table's distribution mode for this write | CommitMetadata provides an interface to add custom metadata to a snapshot summary during a SQL execution, which can be beneficial for purposes such as auditing or change tracking. If properties start with `snapshot-property.`, then that prefix will be removed from each property. Here is an example: diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 31172fb531bd..1cd14dd1888e 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -819,7 +819,7 @@ that provide additional information about the changes being tracked. These colum Here is an example of corresponding results. It shows that the first snapshot inserted 2 records, and the second snapshot deleted 1 record. -| id | name |_change_type | _change_ordinal | _change_snapshot_id | +| id | name |_change_type | _change_ordinal | _commit_snapshot_id | |---|--------|---|---|---| |1 | Alice |INSERT |0 |5390529835796506035| |2 | Bob |INSERT |0 |5390529835796506035| @@ -839,7 +839,7 @@ CALL spark_catalog.system.create_changelog_view( With the net changes, the above changelog view only contains the following row since Alice was inserted in the first snapshot and deleted in the second snapshot. -| id | name |_change_type | _change_ordinal | _change_snapshot_id | +| id | name |_change_type | _change_ordinal | _commit_snapshot_id | |---|--------|---|---|---| |2 | Bob |INSERT |0 |5390529835796506035| 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/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index 96fcc5f7ce32..cc8ca76fe5f7 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -195,16 +195,19 @@ WHERE EXISTS (SELECT oid FROM prod.db.returned_orders WHERE t1.oid = oid) For more complex row-level updates based on incoming data, see the section on `MERGE INTO`. ## Writing to Branches -Branch writes can be performed via SQL by providing a branch identifier, `branch_yourBranch` in the operation. -Branch writes can also be performed as part of a write-audit-publish (WAP) workflow by specifying the `spark.wap.branch` config. -Note WAP branch and branch identifier cannot both be specified. -Also, the branch must exist before performing the write. -The operation does **not** create the branch if it does not exist. -For more information on branches please refer to [branches](branching.md). + +The branch must exist before performing write. Operations do **not** create the branch if it does not exist. +A branch can be created using [Spark DDL](spark-ddl.md#branching-and-tagging-ddl). !!! info Note: When writing to a branch, the current schema of the table will be used for validation. +### Via SQL + +Branch writes can be performed by providing a branch identifier, `branch_yourBranch` in the operation. + +Branch writes can also be performed as part of a write-audit-publish (WAP) workflow by specifying the `spark.wap.branch` config. +Note WAP branch and branch identifier cannot both be specified. ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. @@ -228,6 +231,22 @@ SET spark.wap.branch = audit-branch INSERT INTO prod.db.table VALUES (3, 'c'); ``` +### Via DataFrames + +Branch writes via DataFrames can be performed by providing a branch identifier, `branch_yourBranch` in the operation. + +```scala +// To insert into `audit` branch +val data: DataFrame = ... +data.writeTo("prod.db.table.branch_audit").append() +``` + +```scala +// To overwrite `audit` branch +val data: DataFrame = ... +data.writeTo("prod.db.table.branch_audit").overwritePartitions() +``` + ## Writing with DataFrames Spark 3 introduced the new `DataFrameWriterV2` API for writing to tables using data frames. The v2 API is recommended for several reasons: diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 5e43aa1d26d9..604fede583d8 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -53,7 +53,7 @@ nav: - hive.md - Trino: https://trino.io/docs/current/connector/iceberg.html - Daft: daft.md - - Clickhouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg + - ClickHouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg - Presto: https://prestodb.io/docs/current/connector/iceberg.html - Dremio: https://docs.dremio.com/data-formats/apache-iceberg/ - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog @@ -63,6 +63,7 @@ nav: - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - Doris: https://doris.apache.org/docs/dev/lakehouse/datalake-analytics/iceberg + - Druid: https://druid.apache.org/docs/latest/development/extensions-contrib/iceberg/ - Integrations: - aws.md - dell.md 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/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java deleted file mode 100644 index 157f04b8b0ed..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. - */ -class AggregatedStatistics<D extends DataStatistics<D, S>, S> implements Serializable { - - private final long checkpointId; - private final DataStatistics<D, S> dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } - - AggregatedStatistics(long checkpoint, DataStatistics<D, S> dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; - } - - DataStatistics<D, S> dataStatistics() { - return dataStatistics; - } - - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) - .toString(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java deleted file mode 100644 index e8ff61dbeb27..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.util.Set; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. - */ -class AggregatedStatisticsTracker<D extends DataStatistics<D, S>, S> { - private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; - private final String operatorName; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; - private final int parallelism; - private final Set<Integer> inProgressSubtaskSet; - private volatile AggregatedStatistics<D, S> inProgressStatistics; - - AggregatedStatisticsTracker( - String operatorName, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer, - int parallelism) { - this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; - this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); - } - - AggregatedStatistics<D, S> updateAndCheckCompletion( - int subtask, DataStatisticsEvent<D, S> event) { - long checkpointId = event.checkpointId(); - - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { - LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", - operatorName, - inProgressStatistics.checkpointId(), - checkpointId); - return null; - } - - AggregatedStatistics<D, S> completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); - } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); - } - - inProgressStatistics = null; - inProgressSubtaskSet.clear(); - } - - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); - } - - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); - } - - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); - } - - return completedStatistics; - } - - @VisibleForTesting - AggregatedStatistics<D, S> inProgressStatistics() { - return inProgressStatistics; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java deleted file mode 100644 index 5157a37cf2cd..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ /dev/null @@ -1,197 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; -import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be - * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to - * shuffle record to improve data clustering while maintaining relative balanced traffic - * distribution to downstream subtasks. - */ -@Internal -class DataStatisticsOperator<D extends DataStatistics<D, S>, S> - extends AbstractStreamOperator<DataStatisticsOrRecord<D, S>> - implements OneInputStreamOperator<RowData, DataStatisticsOrRecord<D, S>>, OperatorEventHandler { - - private static final long serialVersionUID = 1L; - - private final String operatorName; - private final RowDataWrapper rowDataWrapper; - private final SortKey sortKey; - private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; - private transient volatile DataStatistics<D, S> localStatistics; - private transient volatile DataStatistics<D, S> globalStatistics; - private transient ListState<DataStatistics<D, S>> globalStatisticsState; - - DataStatisticsOperator( - String operatorName, - Schema schema, - SortOrder sortOrder, - OperatorEventGateway operatorEventGateway, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - this.operatorName = operatorName; - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - this.sortKey = new SortKey(schema, sortOrder); - this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = - context - .getOperatorStateStore() - .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); - - if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - if (globalStatisticsState.get() == null - || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( - "Operator {} subtask {} doesn't have global statistics state to restore", - operatorName, - subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); - } else { - LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); - } - } else { - globalStatistics = statisticsSerializer.createInstance(); - } - } - - @Override - public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - } - - @Override - @SuppressWarnings("unchecked") - public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - Preconditions.checkArgument( - event instanceof DataStatisticsEvent, - String.format( - "Operator %s subtask %s received unexpected operator event %s", - operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent<D, S> statisticsEvent = (DataStatisticsEvent<D, S>) event; - LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", - operatorName, - statisticsEvent.checkpointId()); - globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - - @Override - public void processElement(StreamRecord<RowData> streamRecord) { - RowData record = streamRecord.getValue(); - StructLike struct = rowDataWrapper.wrap(record); - sortKey.wrap(struct); - localStatistics.add(sortKey); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", - operatorName, - checkpointId, - subTaskId); - - // Pass global statistics to partitioners so that all the operators refresh statistics - // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - - // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores - // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { - globalStatisticsState.clear(); - LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); - globalStatisticsState.add(globalStatistics); - } - - // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, - operatorName, - checkpointId, - localStatistics); - - // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); - } - - @VisibleForTesting - DataStatistics<D, S> localDataStatistics() { - return localStatistics; - } - - @VisibleForTesting - DataStatistics<D, S> globalDataStatistics() { - return globalStatistics; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 8716cb872d0e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static <D extends DataStatistics<D, S>, S> byte[] serializeDataStatistics( - DataStatistics<D, S> dataStatistics, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static <D extends DataStatistics<D, S>, S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static <D extends DataStatistics<D, S>, S> byte[] serializeAggregatedStatistics( - AggregatedStatistics<D, S> aggregatedStatistics, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - static <D extends DataStatistics<D, S>, S> - AggregatedStatistics<D, S> deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics<D, S> dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index b6cccd0566fc..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> { - private final MapSerializer<SortKey, Long> mapSerializer; - - static MapDataStatisticsSerializer fromSortKeySerializer( - TypeSerializer<SortKey> sortKeySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer<SortKey, Long> mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> duplicate() { - MapSerializer<SortKey, Long> duplicateMapSerializer = - (MapSerializer<SortKey, Long>) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public MapDataStatistics createInstance() { - return new MapDataStatistics(); - } - - @Override - public MapDataStatistics copy(DataStatistics<MapDataStatistics, Map<SortKey, Long>> obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer<SortKey> keySerializer = mapSerializer.getKeySerializer(); - Map<SortKey, Long> newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry<SortKey, Long> entry : from.statistics().entrySet()) { - SortKey newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics<MapDataStatistics, Map<SortKey, Long>> copy( - DataStatistics<MapDataStatistics, Map<SortKey, Long>> from, - DataStatistics<MapDataStatistics, Map<SortKey, Long>> reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize( - DataStatistics<MapDataStatistics, Map<SortKey, Long>> obj, DataOutputView target) - throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics<MapDataStatistics, Map<SortKey, Long>> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics<MapDataStatistics, Map<SortKey, Long>> deserialize( - DataStatistics<MapDataStatistics, Map<SortKey, Long>> reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics<MapDataStatistics, Map<SortKey, Long>>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer<?>[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer<?>[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer<?>[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer<SortKey, Long> mapSerializer = - (MapSerializer<SortKey, Long>) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java deleted file mode 100644 index dde86b5b6047..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ /dev/null @@ -1,387 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Internal partitioner implementation that supports MapDataStatistics, which is typically used for - * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used - * for high-cardinality use cases. Otherwise, the memory footprint is too high. - * - * <p>It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always - * precise when calculating close cost for every file, target weight per subtask, padding residual - * weight, assigned weight without close cost. - * - * <p>All actions should be executed in a single Flink mailbox thread. So there is no need to make - * it thread safe. - */ -class MapRangePartitioner implements Partitioner<RowData> { - private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); - - private final RowDataWrapper rowDataWrapper; - private final SortKey sortKey; - private final Comparator<StructLike> comparator; - private final Map<SortKey, Long> mapStatistics; - private final double closeFileCostInWeightPercentage; - - // Counter that tracks how many times a new key encountered - // where there is no traffic statistics learned about it. - private long newSortKeyCounter; - private long lastNewSortKeyLogTimeMilli; - - // lazily computed due to the need of numPartitions - private Map<SortKey, KeyAssignment> assignment; - private NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - MapDataStatistics dataStatistics, - double closeFileCostInWeightPercentage) { - dataStatistics - .statistics() - .entrySet() - .forEach( - entry -> - Preconditions.checkArgument( - entry.getValue() > 0, - "Invalid statistics: weight is 0 for key %s", - entry.getKey())); - - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = dataStatistics.statistics(); - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; - this.newSortKeyCounter = 0; - this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); - } - - @Override - public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map<SortKey, KeyAssignment> assignmentMap = assignment(numPartitions); - // reuse the sortKey and rowDataWrapper - sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); - if (keyAssignment == null) { - LOG.trace( - "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", - sortKey); - // Ideally unknownKeyCounter should be published as a counter metric. - // It seems difficult to pass in MetricGroup into the partitioner. - // Just log an INFO message every minute. - newSortKeyCounter += 1; - long now = System.currentTimeMillis(); - if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); - lastNewSortKeyLogTimeMilli = now; - } - return (int) (newSortKeyCounter % numPartitions); - } - - return keyAssignment.select(); - } - - @VisibleForTesting - Map<SortKey, KeyAssignment> assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map<SortKey, Long> mapStatistics() { - return mapStatistics; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map<Integer, Pair<Long, Integer>> assignmentInfo() { - Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map<SortKey, KeyAssignment> buildAssignment( - int numPartitions, - NavigableMap<SortKey, Long> sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map<SortKey, KeyAssignment> assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List<Integer> assignedSubtasks = Lists.newArrayList(); - List<Long> subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List<Integer> assignedSubtasks, - List<Long> subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 45af9241b743..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index 890cc361b246..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Test; - -public class TestAggregatedStatistics { - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - @Test - public void mergeDataStatisticTest() { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - - AggregatedStatistics<MapDataStatistics, Map<SortKey, Long>> aggregatedStatistics = - new AggregatedStatistics<>(1, statisticsSerializer); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(keyA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyB) - + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java deleted file mode 100644 index 4c64ce522201..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ /dev/null @@ -1,178 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; - -public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - private final SortKey keyA = sortKey.copy(); - private final SortKey keyB = sortKey.copy(); - - private AggregatedStatisticsTracker<MapDataStatistics, Map<SortKey, Long>> - aggregatedStatisticsTracker; - - public TestAggregatedStatisticsTracker() { - keyA.set(0, "a"); - keyB.set(0, "b"); - } - - @Before - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } - - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); - } - - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - checkpoint2Subtask0DataStatistic.add(keyB); - checkpoint2Subtask0DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); - } - - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics<MapDataStatistics, Map<SortKey, Long>> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(keyA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyA) - + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); - assertThat((long) globalDataStatistics.statistics().get(keyB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyB) - + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); - // Receive data statistics from all subtasks at checkpoint 2 - completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java deleted file mode 100644 index 3df714059c37..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ /dev/null @@ -1,178 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsCoordinator { - private static final String OPERATOR_NAME = "TestCoordinator"; - private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> - dataStatisticsCoordinator; - - @Before - public void before() throws Exception { - receivingTasks = EventReceivingTasks.createForRunningTasks(); - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); - } - - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - } - - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - } - - @Test - public void testDataStatisticsEventHandling() throws Exception { - tasksReady(); - SortKey key = sortKey.copy(); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify global data statistics is the aggregation of all subtasks data statistics - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - keyA, 2L, - keyB, 3L, - keyC, 5L)); - } - - static void setAllTasksReady( - int subtasks, - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> dataStatisticsCoordinator, - EventReceivingTasks receivingTasks) { - for (int i = 0; i < subtasks; i++) { - dataStatisticsCoordinator.executionAttemptReady( - i, 0, receivingTasks.createGatewayForSubtask(i, 0)); - } - } - - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> coordinator) { - CompletableFuture<Void> future = new CompletableFuture<>(); - coordinator.callInCoordinatorThread( - () -> { - future.complete(null); - return null; - }, - "Coordinator fails to process action"); - - try { - future.get(); - } catch (InterruptedException e) { - throw new AssertionError("test interrupted"); - } catch (ExecutionException e) { - ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java deleted file mode 100644 index 5e0a752be506..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsCoordinatorProvider { - private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsCoordinatorProvider<MapDataStatistics, Map<SortKey, Long>> provider; - private EventReceivingTasks receivingTasks; - - @Before - public void before() { - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); - receivingTasks = EventReceivingTasks.createForRunningTasks(); - } - - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - SortKey keyD = sortKey.copy(); - keyD.set(0, "c"); - SortKey keyE = sortKey.copy(); - keyE.set(0, "c"); - - try (RecreateOnResetOperatorCoordinator coordinator = - (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> dataStatisticsCoordinator = - (DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>>) - coordinator.getInternalCoordinator(); - - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyC); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyD); - checkpoint2Subtask0DataStatistic.add(keyE); - checkpoint2Subtask0DataStatistic.add(keyE); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); - - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); - } - } - - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> coordinator) - throws InterruptedException, ExecutionException { - CompletableFuture<byte[]> future = new CompletableFuture<>(); - coordinator.checkpointCoordinator(checkpointId, future); - return future.get(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java deleted file mode 100644 index 0e99a2d74ccb..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; -import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; -import org.apache.flink.streaming.util.MockOutput; -import org.apache.flink.streaming.util.MockStreamConfig; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsOperator { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final RowType rowType = RowType.of(new VarCharType(), new IntType()); - private final TypeSerializer<RowData> rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> - statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer( - new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } - - @Before - public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask<String, String>(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); - } - - private DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> createOperator() { - MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return new DataStatisticsOperator<>( - "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); - } - - @After - public void clean() throws Exception { - operator.close(); - } - - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness = createHarness(this.operator)) { - StateInitializationContext stateContext = getStateContext(); - operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - Map<SortKey, Long> expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); - - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map<SortKey, Long> statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); - - testHarness.endInput(); - } - } - - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness = createHarness(this.operator)) { - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - - List<RowData> recordsOutput = - testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) - .collect(Collectors.toList()); - assertThat(recordsOutput) - .containsExactlyInAnyOrderElementsOf( - ImmutableList.of( - GenericRowData.of(StringData.fromString("a"), 2), - GenericRowData.of(StringData.fromString("b"), 3), - GenericRowData.of(StringData.fromString("b"), 1))); - } - } - - @Test - public void testRestoreState() throws Exception { - OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness1 = createHarness(this.operator)) { - MapDataStatistics mapDataStatistics = new MapDataStatistics(); - - SortKey key = sortKey.copy(); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "b"); - mapDataStatistics.add(key); - key.set(0, "c"); - mapDataStatistics.add(key); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map<SortKey, Long> expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> event = - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); - operator.handleOperatorEvent(event); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf(expectedMap); - snapshot = testHarness1.snapshot(1L, 0); - } - - // Use the snapshot to initialize state for another new operator and then verify that the global - // statistics for the new operator is same as before - DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { - testHarness2.setup(); - testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - - // restored RowData is BinaryRowData. convert to GenericRowData for comparison - Map<SortKey, Long> restoredStatistics = Maps.newHashMap(); - restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map<SortKey, Long> expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); - } - } - - private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); - AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); - CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); - OperatorStateStore operatorStateStore = - abstractStateBackend.createOperatorStateBackend( - env, "test-operator", Collections.emptyList(), cancelStreamRegistry); - return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); - } - - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - createHarness( - final DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); - harness.open(); - return harness; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java deleted file mode 100644 index be2beeebc93c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; - -public class TestMapDataStatistics { - private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); - private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private final RowDataWrapper rowWrapper = - new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); - - @Test - public void testAddsAndGet() { - MapDataStatistics dataStatistics = new MapDataStatistics(); - - GenericRowData reusedRow = - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("c")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("a")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - Map<SortKey, Long> actual = dataStatistics.statistics(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyA = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyB = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyC = sortKey.copy(); - - Map<SortKey, Long> expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); - assertThat(actual).isEqualTo(expected); - } -} diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index afd4c3c6e114..aac01c9c6931 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -66,6 +66,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } + implementation libs.datasketches + testImplementation libs.flink118.connector.test.utils testImplementation libs.flink118.core testImplementation libs.flink118.runtime @@ -117,7 +119,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core - testImplementation libs.junit.vintage.engine } test { diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 3b2c74fd6ece..a9ad386a5a4a 100644 --- a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.nio.charset.StandardCharsets; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -28,6 +29,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -67,6 +70,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator<StructLike> SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -83,10 +88,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List<Integer> keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index 1453753849ec..9b0c7a938920 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -70,6 +70,7 @@ public class FlinkCatalogFactory implements CatalogFactory { public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; public static final String DEFAULT_DATABASE = "default-database"; public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String DEFAULT_CATALOG_NAME = "default_catalog"; public static final String BASE_NAMESPACE = "base-namespace"; public static final String TYPE = "type"; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public <E extends Enum<E>> EnumConfParser<E> enumConfParser(Class<E> enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser<DoubleConfParser, Double> { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser<StringConfParser, String> { private String defaultValue; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption<String> DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption<String> RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption<Double> RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption<String> BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); 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/FlinkSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 769af7d77140..5cd43a46de37 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -53,15 +53,20 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; +import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -233,15 +238,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + * <p>Default is {@link StatisticsType#Auto} where initially Map statistics is used. But if + * cardinality is higher than the threshold (currently 10K) as defined in {@code + * SketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD}, statistics collection automatically switches to + * the sketch reservoir sampling. + * + * <p>Explicit set the statistics type if the default behavior doesn't work. + * + * @param type to specify the statistics type for range distribution. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rangeDistributionStatisticsType(StatisticsType type) { + if (type != null) { + writeOptions.put(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key(), type.name()); + } + return this; + } + + /** + * If sort order contains partition columns, each sort key would map to one partition and data + * file. This relative weight can avoid placing too many small files for sort keys with low + * traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means + * each key has a base weight of `2%` of the targeted traffic weight per writer task. + * + * <p>E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream + * contains events from now up to 180 days ago. With event time, traffic weight distribution + * across different days typically has a long tail pattern. Current day contains the most + * traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism + * is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer + * task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, + * the range partitioner would put all the oldest 150 days in one writer task. That writer task + * would write to 150 small files (one per day). Keeping 150 open files can potentially consume + * large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time + * can also be potentially slow. If this config is set to `0.02`. It means every sort key has a + * base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially + * avoid placing more than `50` data files (one per day) on one writer task no matter how small + * they are. + * + * <p>This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For + * {@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as + * partition columns. Otherwise, too many partitions and small files may be generated during + * write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. + * + * <p>Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -349,18 +407,20 @@ private <T> DataStreamSink<T> chainIcebergOperators() { // Find out the equality field id list based on the user-provided equality field column names. List<Integer> equalityFieldIds = checkAndGetEqualityFieldIds(); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream<RowData> distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator<WriteResult> writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + SingleOutputStreamOperator<FlinkWriteResult> writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -426,7 +486,7 @@ private <T> DataStreamSink<T> appendDummySink( } private SingleOutputStreamOperator<Void> appendCommitter( - SingleOutputStreamOperator<WriteResult> writerStream) { + SingleOutputStreamOperator<FlinkWriteResult> writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -446,8 +506,11 @@ private SingleOutputStreamOperator<Void> appendCommitter( return committerStream; } - private SingleOutputStreamOperator<WriteResult> appendWriter( - DataStream<RowData> input, RowType flinkRowType, List<Integer> equalityFieldIds) { + private SingleOutputStreamOperator<FlinkWriteResult> appendWriter( + DataStream<RowData> input, + RowType flinkRowType, + List<Integer> equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +544,13 @@ private SingleOutputStreamOperator<WriteResult> appendWriter( IcebergStreamWriter<RowData> streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator<WriteResult> writerStream = + SingleOutputStreamOperator<FlinkWriteResult> writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +560,15 @@ private SingleOutputStreamOperator<WriteResult> appendWriter( private DataStream<RowData> distributeDataStream( DataStream<RowData> input, List<Integer> equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +610,52 @@ private DataStream<RowData> distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator<StatisticsOrRecord> shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +670,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..7108c2008341 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -63,7 +63,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator<Void> - implements OneInputStreamOperator<WriteResult, Void>, BoundedOneInput { + implements OneInputStreamOperator<FlinkWriteResult, Void>, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -96,7 +96,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator<Void> // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List<WriteResult> writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map<Long, List<WriteResult>> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -212,7 +212,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -220,8 +221,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -426,30 +425,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord<WriteResult> element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord<FlinkWriteResult> element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List<WriteResult> writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry<Long, List<WriteResult>> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List<WriteResult> writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..bb5efe982ee1 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter<T> extends AbstractStreamOperator<WriteResult> - implements OneInputStreamOperator<T, WriteResult>, BoundedOneInput { +class IcebergStreamWriter<T> extends AbstractStreamOperator<FlinkWriteResult> + implements OneInputStreamOperator<T, FlinkWriteResult>, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory<T> taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,7 +90,7 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override @@ -102,7 +103,7 @@ public String toString() { } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java deleted file mode 100644 index 157f04b8b0ed..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. - */ -class AggregatedStatistics<D extends DataStatistics<D, S>, S> implements Serializable { - - private final long checkpointId; - private final DataStatistics<D, S> dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } - - AggregatedStatistics(long checkpoint, DataStatistics<D, S> dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; - } - - DataStatistics<D, S> dataStatistics() { - return dataStatistics; - } - - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) - .toString(); - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index e8ff61dbeb27..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 @@ -18,116 +18,245 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import java.util.NavigableMap; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.datasketches.sampling.ReservoirItemsUnion; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. */ -class AggregatedStatisticsTracker<D extends DataStatistics<D, S>, S> { +class AggregatedStatisticsTracker { private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; + private final String operatorName; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; private final int parallelism; - private final Set<Integer> inProgressSubtaskSet; - private volatile AggregatedStatistics<D, S> inProgressStatistics; + private final TypeSerializer<DataStatistics> statisticsSerializer; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final int switchToSketchThreshold; + private final NavigableMap<Long, Aggregation> aggregationsPerCheckpoint; + + private CompletedStatistics completedStatistics; AggregatedStatisticsTracker( String operatorName, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer, - int parallelism) { + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable CompletedStatistics restoredStatistics) { this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); + this.statisticsSerializer = + new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.switchToSketchThreshold = switchToSketchThreshold; + this.completedStatistics = restoredStatistics; + + this.aggregationsPerCheckpoint = Maps.newTreeMap(); } - AggregatedStatistics<D, S> updateAndCheckCompletion( - int subtask, DataStatisticsEvent<D, S> event) { + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { long checkpointId = event.checkpointId(); + LOG.debug( + "Handling statistics event from subtask {} of operator {} for checkpoint {}", + subtask, + operatorName, + checkpointId); - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { + if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " + + "Was expecting data statistics from checkpoint higher than {}", operatorName, - inProgressStatistics.checkpointId(), - checkpointId); + subtask, + checkpointId, + completedStatistics.checkpointId()); return null; } - AggregatedStatistics<D, S> completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, + Aggregation aggregation = + aggregationsPerCheckpoint.computeIfAbsent( checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); + ignored -> + new Aggregation( + parallelism, + downstreamParallelism, + switchToSketchThreshold, + statisticsType, + StatisticsUtil.collectType(statisticsType, completedStatistics))); + DataStatistics dataStatistics = + StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); + if (!aggregation.merge(subtask, dataStatistics)) { + LOG.debug( + "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } + + if (aggregation.isComplete()) { + this.completedStatistics = aggregation.completedStatistics(checkpointId); + // clean up aggregations up to the completed checkpoint id + aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); + return completedStatistics; + } + + return null; + } + + @VisibleForTesting + NavigableMap<Long, Aggregation> aggregationsPerCheckpoint() { + return aggregationsPerCheckpoint; + } + + static class Aggregation { + private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); + + private final Set<Integer> subtaskSet; + private final int parallelism; + private final int downstreamParallelism; + private final int switchToSketchThreshold; + private final StatisticsType configuredType; + private StatisticsType currentType; + private Map<SortKey, Long> mapStatistics; + private ReservoirItemsUnion<SortKey> sketchStatistics; + + Aggregation( + int parallelism, + int downstreamParallelism, + int switchToSketchThreshold, + StatisticsType configuredType, + StatisticsType currentType) { + this.subtaskSet = Sets.newHashSet(); + this.parallelism = parallelism; + this.downstreamParallelism = downstreamParallelism; + this.switchToSketchThreshold = switchToSketchThreshold; + this.configuredType = configuredType; + this.currentType = currentType; + + if (currentType == StatisticsType.Map) { + this.mapStatistics = Maps.newHashMap(); + this.sketchStatistics = null; } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); + this.mapStatistics = null; + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); } + } - inProgressStatistics = null; - inProgressSubtaskSet.clear(); + @VisibleForTesting + Set<Integer> subtaskSet() { + return subtaskSet; } - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + StatisticsType currentType() { + return currentType; } - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); + @VisibleForTesting + Map<SortKey, Long> mapStatistics() { + return mapStatistics; } - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + ReservoirItemsUnion<SortKey> sketchStatistics() { + return sketchStatistics; } - return completedStatistics; - } + private boolean isComplete() { + return subtaskSet.size() == parallelism; + } - @VisibleForTesting - AggregatedStatistics<D, S> inProgressStatistics() { - return inProgressStatistics; + /** @return false if duplicate */ + private boolean merge(int subtask, DataStatistics taskStatistics) { + if (subtaskSet.contains(subtask)) { + return false; + } + + subtaskSet.add(subtask); + merge(taskStatistics); + return true; + } + + @SuppressWarnings("unchecked") + private void merge(DataStatistics taskStatistics) { + if (taskStatistics.type() == StatisticsType.Map) { + Map<SortKey, Long> taskMapStats = (Map<SortKey, Long>) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); + if (configuredType == StatisticsType.Auto + && mapStatistics.size() > switchToSketchThreshold) { + convertCoordinatorToSketch(); + } + } else { + // convert task stats to sketch first + ReservoirItemsSketch<SortKey> taskSketch = + ReservoirItemsSketch.newInstance( + SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); + SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); + sketchStatistics.update(taskSketch); + } + } else { + ReservoirItemsSketch<SortKey> taskSketch = + (ReservoirItemsSketch<SortKey>) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + // convert global stats to sketch first + convertCoordinatorToSketch(); + } + + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } + } + } + + private void convertCoordinatorToSketch() { + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); + this.currentType = StatisticsType.Sketch; + this.mapStatistics = null; + } + + private CompletedStatistics completedStatistics(long checkpointId) { + if (currentType == StatisticsType.Map) { + LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + } else { + ReservoirItemsSketch<SortKey> sketch = sketchStatistics.getResult(); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } + } + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java similarity index 56% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index 8a91411c1d21..e4cba174f0f2 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -18,46 +18,39 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.io.Serializable; import java.util.Arrays; import java.util.Map; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Objects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). */ -class AggregatedStatistics implements Serializable { +class CompletedStatistics { private final long checkpointId; private final StatisticsType type; private final Map<SortKey, Long> keyFrequency; - private final SortKey[] rangeBounds; + private final SortKey[] keySamples; - AggregatedStatistics( + static CompletedStatistics fromKeyFrequency(long checkpointId, Map<SortKey, Long> stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( long checkpointId, StatisticsType type, Map<SortKey, Long> keyFrequency, - SortKey[] rangeBounds) { - Preconditions.checkArgument( - (keyFrequency != null && rangeBounds == null) - || (keyFrequency == null && rangeBounds != null), - "Invalid key frequency or range bounds: both are non-null or null"); + SortKey[] keySamples) { this.checkpointId = checkpointId; this.type = type; this.keyFrequency = keyFrequency; - this.rangeBounds = rangeBounds; - } - - static AggregatedStatistics fromKeyFrequency(long checkpointId, Map<SortKey, Long> stats) { - return new AggregatedStatistics(checkpointId, StatisticsType.Map, stats, null); - } - - static AggregatedStatistics fromRangeBounds(long checkpointId, SortKey[] stats) { - return new AggregatedStatistics(checkpointId, StatisticsType.Sketch, null, stats); + this.keySamples = keySamples; } @Override @@ -66,7 +59,7 @@ public String toString() { .add("checkpointId", checkpointId) .add("type", type) .add("keyFrequency", keyFrequency) - .add("rangeBounds", rangeBounds) + .add("keySamples", keySamples) .toString(); } @@ -76,20 +69,24 @@ public boolean equals(Object o) { return true; } - if (!(o instanceof AggregatedStatistics)) { + if (!(o instanceof CompletedStatistics)) { return false; } - AggregatedStatistics other = (AggregatedStatistics) o; - return Objects.equal(checkpointId, other.checkpointId()) - && Objects.equal(type, other.type()) + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) && Objects.equal(keyFrequency, other.keyFrequency()) - && Arrays.equals(rangeBounds, other.rangeBounds()); + && Arrays.equals(keySamples, other.keySamples()); } @Override public int hashCode() { - return Objects.hashCode(checkpointId, type, keyFrequency, rangeBounds); + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; } StatisticsType type() { @@ -100,11 +97,15 @@ Map<SortKey, Long> keyFrequency() { return keyFrequency; } - SortKey[] rangeBounds() { - return rangeBounds; + SortKey[] keySamples() { + return keySamples; } - long checkpointId() { - return checkpointId; + 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/CompletedStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java new file mode 100644 index 000000000000..7f55188e7f8c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; + +class CompletedStatisticsSerializer extends TypeSerializer<CompletedStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final MapSerializer<SortKey, Long> keyFrequencySerializer; + private final ListSerializer<SortKey> keySamplesSerializer; + + CompletedStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<CompletedStatistics> duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); + } + + @Override + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + keyFrequencySerializer.serialize(record.keyFrequency(), target); + } else { + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); + } + } + + @Override + public CompletedStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + Map<SortKey, Long> keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); + } else { + List<SortKey> sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); + } + } + + @Override + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<CompletedStatistics> snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); + } + + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<CompletedStatistics, CompletedStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public CompletedStatisticsSerializerSnapshot() { + super(CompletedStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers( + CompletedStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new CompletedStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 9d7cf179ab1c..76c59cd5f4b8 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; @@ -29,29 +31,18 @@ * (sketching) can be used. */ @Internal -interface DataStatistics<D extends DataStatistics<D, S>, S> { +interface DataStatistics { + + StatisticsType type(); - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ boolean isEmpty(); /** Add row sortKey to data statistics. */ void add(SortKey sortKey); /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics + * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link + * ReservoirItemsSketch} (high cardinality) */ - S statistics(); + Object result(); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index c8ac79c61bf6..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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -35,60 +36,99 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.ThrowableCatchingRunnable; import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. + * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} + * every subtask and then merge them together. Once aggregation for all subtasks data statistics + * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link + * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the + * aggregated data statistics to improve data clustering. */ @Internal -class DataStatisticsCoordinator<D extends DataStatistics<D, S>, S> implements OperatorCoordinator { +class DataStatisticsCoordinator implements OperatorCoordinator { private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); private final String operatorName; + private final OperatorCoordinator.Context context; + private final Schema schema; + private final SortOrder sortOrder; + private final Comparator<StructLike> comparator; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; + private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; - private final transient AggregatedStatisticsTracker<D, S> aggregatedStatisticsTracker; - private volatile AggregatedStatistics<D, S> completedStatistics; - private volatile boolean started; + private final TypeSerializer<CompletedStatistics> completedStatisticsSerializer; + private final TypeSerializer<GlobalStatistics> globalStatisticsSerializer; + + private transient boolean started; + private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; DataStatisticsCoordinator( String operatorName, OperatorCoordinator.Context context, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { this.operatorName = operatorName; + this.context = context; + this.schema = schema; + this.sortOrder = sortOrder; + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; + this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker( + operatorName, + context.currentParallelism(), + schema, + sortOrder, + downstreamParallelism, + statisticsType, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + completedStatistics); } @Override public void close() throws Exception { coordinatorExecutor.shutdown(); + this.aggregatedStatisticsTracker = null; + this.started = false; LOG.info("Closed data statistics coordinator: {}.", operatorName); } @@ -148,7 +188,7 @@ private void runInCoordinatorThread(ThrowingRunnable<Throwable> action, String a operatorName, actionString, t); - operatorCoordinatorContext.failJob(t); + context.failJob(t); } }); } @@ -157,42 +197,108 @@ private void ensureStarted() { Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); } - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent<D, S> event) { - AggregatedStatistics<D, S> aggregatedStatistics = + private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { + CompletedStatistics maybeCompletedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + if (maybeCompletedStatistics != null) { + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator<StructLike> comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics<D, S> globalDataStatistics) { - callInCoordinatorThread( + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( () -> { - DataStatisticsEvent<D, S> dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); + for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference + subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } - - return null; }, String.format( "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } } @Override - @SuppressWarnings("unchecked") public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { runInCoordinatorThread( () -> { @@ -202,8 +308,14 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent<D, S>) event)); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -218,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); + 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)); } @@ -229,12 +346,10 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r public void notifyCheckpointComplete(long checkpointId) {} @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, @@ -244,8 +359,13 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + this.completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); } @Override @@ -269,7 +389,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", subtask, attemptNumber, operatorName); @@ -295,14 +415,20 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics<D, S> completedStatistics() { + CompletedStatistics completedStatistics() { return completedStatistics; } + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + private static class SubtaskGateways { private final String operatorName; private final Map<Integer, SubtaskGateway>[] gateways; + @SuppressWarnings("unchecked") private SubtaskGateways(String operatorName, int parallelism) { this.operatorName = operatorName; gateways = new Map[parallelism]; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java index 47dbfc3cfbe1..9d7d989c298e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -19,33 +19,52 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; /** * DataStatisticsCoordinatorProvider provides the method to create new {@link * DataStatisticsCoordinator} */ @Internal -public class DataStatisticsCoordinatorProvider<D extends DataStatistics<D, S>, S> - extends RecreateOnResetOperatorCoordinator.Provider { +public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { private final String operatorName; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; public DataStatisticsCoordinatorProvider( String operatorName, OperatorID operatorID, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { super(operatorID); this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + return new DataStatisticsCoordinator( + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 5157a37cf2cd..59c38b239725 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -47,9 +48,8 @@ * distribution to downstream subtasks. */ @Internal -class DataStatisticsOperator<D extends DataStatistics<D, S>, S> - extends AbstractStreamOperator<DataStatisticsOrRecord<D, S>> - implements OneInputStreamOperator<RowData, DataStatisticsOrRecord<D, S>>, OperatorEventHandler { +public class DataStatisticsOperator extends AbstractStreamOperator<StatisticsOrRecord> + implements OneInputStreamOperator<RowData, StatisticsOrRecord>, OperatorEventHandler { private static final long serialVersionUID = 1L; @@ -57,141 +57,209 @@ class DataStatisticsOperator<D extends DataStatistics<D, S>, S> private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; - private transient volatile DataStatistics<D, S> localStatistics; - private transient volatile DataStatistics<D, S> globalStatistics; - private transient ListState<DataStatistics<D, S>> globalStatisticsState; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final TypeSerializer<DataStatistics> taskStatisticsSerializer; + private final TypeSerializer<GlobalStatistics> globalStatisticsSerializer; + + private transient int parallelism; + private transient int subtaskIndex; + private transient ListState<GlobalStatistics> globalStatisticsState; + // current statistics type may be different from the config due to possible + // migration from Map statistics to Sketch statistics when high cardinality detected + private transient volatile StatisticsType taskStatisticsType; + private transient volatile DataStatistics localStatistics; + private transient volatile GlobalStatistics globalStatistics; DataStatisticsOperator( String operatorName, Schema schema, SortOrder sortOrder, OperatorEventGateway operatorEventGateway, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { + int downstreamParallelism, + StatisticsType statisticsType) { this.operatorName = operatorName; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = + this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); + this.subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + + // Use union state so that new subtasks can also restore global statistics during scale-up. + this.globalStatisticsState = context .getOperatorStateStore() .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); if (globalStatisticsState.get() == null || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( + LOG.info( "Operator {} subtask {} doesn't have global statistics state to restore", operatorName, subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; } - } else { - globalStatistics = statisticsSerializer.createInstance(); + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); } + + this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); + this.localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); } @Override public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } } @Override - @SuppressWarnings("unchecked") public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); Preconditions.checkArgument( - event instanceof DataStatisticsEvent, + event instanceof StatisticsEvent, String.format( "Operator %s subtask %s received unexpected operator event %s", operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent<D, S> statisticsEvent = (DataStatisticsEvent<D, S>) event; + StatisticsEvent statisticsEvent = (StatisticsEvent) event; LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", + "Operator {} subtask {} received global data event from coordinator checkpoint {}", operatorName, + subtaskIndex, statisticsEvent.checkpointId()); - globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); + checkStatisticsTypeMigration(); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } } @Override public void processElement(StreamRecord<RowData> streamRecord) { + // collect data statistics RowData record = streamRecord.getValue(); StructLike struct = rowDataWrapper.wrap(record); sortKey.wrap(struct); localStatistics.add(sortKey); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); + + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", + "Operator {} subtask {} snapshotting data statistics for checkpoint {}", operatorName, - checkpointId, - subTaskId); + subtaskIndex, + checkpointId); - // Pass global statistics to partitioners so that all the operators refresh statistics + // Pass global statistics to partitioner so that all the operators refresh statistics // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { + if (globalStatistics != null && getRuntimeContext().getIndexOfThisSubtask() == 0) { globalStatisticsState.clear(); LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); + "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); globalStatisticsState.add(globalStatistics); + LOG.debug( + "Operator {} subtask {} saved global statistics to state: {}", + operatorName, + subtaskIndex, + globalStatistics); } // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, + LOG.info( + "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", operatorName, - checkpointId, - localStatistics); + subtaskIndex, + checkpointId); + operatorEventGateway.sendEventToCoordinator( + StatisticsEvent.createTaskStatisticsEvent( + checkpointId, localStatistics, taskStatisticsSerializer)); // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); + localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @SuppressWarnings("unchecked") + private void checkStatisticsTypeMigration() { + // only check if the statisticsType config is Auto and localStatistics is currently Map type + if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { + Map<SortKey, Long> mapStatistics = (Map<SortKey, Long>) localStatistics.result(); + // convert if local statistics has cardinality over the threshold or + // if received global statistics is already sketch type + if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { + LOG.info( + "Operator {} subtask {} switched local statistics from Map to Sketch.", + operatorName, + subtaskIndex); + this.taskStatisticsType = StatisticsType.Sketch; + this.localStatistics = + StatisticsUtil.createTaskStatistics( + taskStatisticsType, parallelism, downstreamParallelism); + SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); + } + } } @VisibleForTesting - DataStatistics<D, S> localDataStatistics() { + DataStatistics localStatistics() { return localStatistics; } @VisibleForTesting - DataStatistics<D, S> globalDataStatistics() { + GlobalStatistics globalStatistics() { return globalStatistics; } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory<StatisticsOrRecord> + implements CoordinatedOperatorFactory<StatisticsOrRecord>, + OneInputStreamOperatorFactory<RowData, StatisticsOrRecord> { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public <T extends StreamOperator<StatisticsOrRecord>> T createStreamOperator( + StreamOperatorParameters<StatisticsOrRecord> parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java new file mode 100644 index 000000000000..c25481b3c1f2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class DataStatisticsSerializer extends TypeSerializer<DataStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final MapSerializer<SortKey, Long> mapSerializer; + private final SortKeySketchSerializer sketchSerializer; + + DataStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer<DataStatistics> duplicate() { + TypeSerializer<SortKey> duplicateSortKeySerializer = sortKeySerializer.duplicate(); + return (duplicateSortKeySerializer == sortKeySerializer) + ? this + : new DataStatisticsSerializer(duplicateSortKeySerializer); + } + + @Override + public DataStatistics createInstance() { + return new MapDataStatistics(); + } + + @SuppressWarnings("unchecked") + @Override + public DataStatistics copy(DataStatistics obj) { + StatisticsType statisticsType = obj.type(); + if (statisticsType == StatisticsType.Map) { + MapDataStatistics from = (MapDataStatistics) obj; + Map<SortKey, Long> fromStats = (Map<SortKey, Long>) from.result(); + Map<SortKey, Long> toStats = Maps.newHashMap(fromStats); + return new MapDataStatistics(toStats); + } else if (statisticsType == StatisticsType.Sketch) { + // because ReservoirItemsSketch doesn't expose enough public methods for cloning, + // this implementation adopted the less efficient serialization and deserialization. + SketchDataStatistics from = (SketchDataStatistics) obj; + ReservoirItemsSketch<SortKey> fromStats = (ReservoirItemsSketch<SortKey>) from.result(); + byte[] bytes = fromStats.toByteArray(sketchSerializer); + Memory memory = Memory.wrap(bytes); + ReservoirItemsSketch<SortKey> toStats = + ReservoirItemsSketch.heapify(memory, sketchSerializer); + return new SketchDataStatistics(toStats); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics copy(DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @SuppressWarnings("unchecked") + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + StatisticsType statisticsType = obj.type(); + statisticsTypeSerializer.serialize(obj.type(), target); + if (statisticsType == StatisticsType.Map) { + Map<SortKey, Long> mapStatistics = (Map<SortKey, Long>) obj.result(); + mapSerializer.serialize(mapStatistics, target); + } else if (statisticsType == StatisticsType.Sketch) { + ReservoirItemsSketch<SortKey> sketch = (ReservoirItemsSketch<SortKey>) obj.result(); + byte[] sketchBytes = sketch.toByteArray(sketchSerializer); + target.writeInt(sketchBytes.length); + target.write(sketchBytes); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataInputView source) throws IOException { + StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); + if (statisticsType == StatisticsType.Map) { + Map<SortKey, Long> mapStatistics = mapSerializer.deserialize(source); + return new MapDataStatistics(mapStatistics); + } else if (statisticsType == StatisticsType.Sketch) { + int numBytes = source.readInt(); + byte[] sketchBytes = new byte[numBytes]; + source.read(sketchBytes); + Memory sketchMemory = Memory.wrap(sketchBytes); + ReservoirItemsSketch<SortKey> sketch = + ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); + return new SketchDataStatistics(sketch); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsSerializer)) { + return false; + } + + DataStatisticsSerializer other = (DataStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<DataStatistics> snapshotConfiguration() { + return new DataStatisticsSerializerSnapshot(this); + } + + public static class DataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<DataStatistics, DataStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DataStatisticsSerializerSnapshot() { + super(DataStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new DataStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 8716cb872d0e..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static <D extends DataStatistics<D, S>, S> byte[] serializeDataStatistics( - DataStatistics<D, S> dataStatistics, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static <D extends DataStatistics<D, S>, S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static <D extends DataStatistics<D, S>, S> byte[] serializeAggregatedStatistics( - AggregatedStatistics<D, S> aggregatedStatistics, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - static <D extends DataStatistics<D, S>, S> - AggregatedStatistics<D, S> deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer<DataStatistics<D, S>> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics<D, S> dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..dfb947a84a0c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer<GlobalStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final ListSerializer<SortKey> rangeBoundsSerializer; + private final ListSerializer<Integer> intsSerializer; + private final ListSerializer<Long> longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<GlobalStatistics> duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry<SortKey, KeyAssignment> entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map<SortKey, KeyAssignment> keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List<Integer> assignedSubtasks = intsSerializer.deserialize(source); + List<Long> subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List<SortKey> sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<GlobalStatistics> snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<GlobalStatistics, GlobalStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() { + super(GlobalStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..781bcc646023 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List<Integer> assignedSubtasks; + private final List<Long> subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List<Integer> assignedSubtasks, + List<Long> subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List<Integer> assignedSubtasks() { + return assignedSubtasks; + } + + List<Long> subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** + * Select a subtask for the key. + * + * @return subtask id + */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..9d8167460a1b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map<SortKey, KeyAssignment> keyAssignments; + + MapAssignment(int numPartitions, Map<SortKey, KeyAssignment> keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map<SortKey, Long> mapStatistics, + double closeFileCostWeightPercentage, + Comparator<StructLike> comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map<SortKey, KeyAssignment> keyAssignments() { + return keyAssignments; + } + + /** + * Returns assignment summary for every subtask. + * + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map<Integer, Pair<Long, Integer>> assignmentInfo() { + Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map<SortKey, KeyAssignment> assignment( + int numPartitions, + Map<SortKey, Long> mapStatistics, + double closeFileCostWeightPercentage, + Comparator<StructLike> comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map<SortKey, KeyAssignment> buildAssignment( + int numPartitions, + NavigableMap<SortKey, Long> sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map<SortKey, KeyAssignment> assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List<Integer> assignedSubtasks = Lists.newArrayList(); + List<Long> subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 0b63e2721178..05b943f6046f 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -19,52 +19,70 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics<MapDataStatistics, Map<SortKey, Long>> { - private final Map<SortKey, Long> statistics; +class MapDataStatistics implements DataStatistics { + private final Map<SortKey, Long> keyFrequency; MapDataStatistics() { - this.statistics = Maps.newHashMap(); + this.keyFrequency = Maps.newHashMap(); } - MapDataStatistics(Map<SortKey, Long> statistics) { - this.statistics = statistics; + MapDataStatistics(Map<SortKey, Long> keyFrequency) { + this.keyFrequency = keyFrequency; + } + + @Override + public StatisticsType type() { + return StatisticsType.Map; } @Override public boolean isEmpty() { - return statistics.isEmpty(); + return keyFrequency.isEmpty(); } @Override public void add(SortKey sortKey) { - if (statistics.containsKey(sortKey)) { - statistics.merge(sortKey, 1L, Long::sum); + if (keyFrequency.containsKey(sortKey)) { + keyFrequency.merge(sortKey, 1L, Long::sum); } else { // clone the sort key before adding to map because input sortKey object can be reused SortKey copiedKey = sortKey.copy(); - statistics.put(copiedKey, 1L); + keyFrequency.put(copiedKey, 1L); } } @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + public Object result() { + return keyFrequency; } @Override - public Map<SortKey, Long> statistics() { - return statistics; + public String toString() { + return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); } @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof MapDataStatistics)) { + return false; + } + + MapDataStatistics other = (MapDataStatistics) o; + return Objects.equal(keyFrequency, other.keyFrequency); + } + + @Override + public int hashCode() { + return Objects.hashCode(keyFrequency); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index b6cccd0566fc..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> { - private final MapSerializer<SortKey, Long> mapSerializer; - - static MapDataStatisticsSerializer fromSortKeySerializer( - TypeSerializer<SortKey> sortKeySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer<SortKey, Long> mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> duplicate() { - MapSerializer<SortKey, Long> duplicateMapSerializer = - (MapSerializer<SortKey, Long>) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public MapDataStatistics createInstance() { - return new MapDataStatistics(); - } - - @Override - public MapDataStatistics copy(DataStatistics<MapDataStatistics, Map<SortKey, Long>> obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer<SortKey> keySerializer = mapSerializer.getKeySerializer(); - Map<SortKey, Long> newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry<SortKey, Long> entry : from.statistics().entrySet()) { - SortKey newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics<MapDataStatistics, Map<SortKey, Long>> copy( - DataStatistics<MapDataStatistics, Map<SortKey, Long>> from, - DataStatistics<MapDataStatistics, Map<SortKey, Long>> reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize( - DataStatistics<MapDataStatistics, Map<SortKey, Long>> obj, DataOutputView target) - throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics<MapDataStatistics, Map<SortKey, Long>> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics<MapDataStatistics, Map<SortKey, Long>> deserialize( - DataStatistics<MapDataStatistics, Map<SortKey, Long>> reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics<MapDataStatistics, Map<SortKey, Long>>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer<?>[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer<?>[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer<?>[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer<SortKey, Long> mapSerializer = - (MapSerializer<SortKey, Long>) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index dde86b5b6047..f36a078c94e0 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -18,29 +18,14 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,52 +46,28 @@ class MapRangePartitioner implements Partitioner<RowData> { private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; - private final Comparator<StructLike> comparator; - private final Map<SortKey, Long> mapStatistics; - private final double closeFileCostInWeightPercentage; + private final MapAssignment mapAssignment; // Counter that tracks how many times a new key encountered // where there is no traffic statistics learned about it. private long newSortKeyCounter; private long lastNewSortKeyLogTimeMilli; - // lazily computed due to the need of numPartitions - private Map<SortKey, KeyAssignment> assignment; - private NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - MapDataStatistics dataStatistics, - double closeFileCostInWeightPercentage) { - dataStatistics - .statistics() - .entrySet() - .forEach( - entry -> - Preconditions.checkArgument( - entry.getValue() > 0, - "Invalid statistics: weight is 0 for key %s", - entry.getKey())); - + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = dataStatistics.statistics(); - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.mapAssignment = mapAssignment; this.newSortKeyCounter = 0; this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); } @Override public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map<SortKey, KeyAssignment> assignmentMap = assignment(numPartitions); // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; if (keyAssignment == null) { LOG.trace( "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", @@ -117,271 +78,18 @@ public int partition(RowData row, int numPartitions) { newSortKeyCounter += 1; long now = System.currentTimeMillis(); if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + LOG.info( + "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", + newSortKeyCounter); lastNewSortKeyLogTimeMilli = now; + newSortKeyCounter = 0; } - return (int) (newSortKeyCounter % numPartitions); + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); } - return keyAssignment.select(); - } - - @VisibleForTesting - Map<SortKey, KeyAssignment> assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map<SortKey, Long> mapStatistics() { - return mapStatistics; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map<Integer, Pair<Long, Integer>> assignmentInfo() { - Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map<SortKey, KeyAssignment> buildAssignment( - int numPartitions, - NavigableMap<SortKey, Long> sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map<SortKey, KeyAssignment> assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List<Integer> assignedSubtasks = Lists.newArrayList(); - List<Long> subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List<Integer> assignedSubtasks, - List<Long> subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java new file mode 100644 index 000000000000..6608b938f5a8 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ +@Internal +public class RangePartitioner implements Partitioner<StatisticsOrRecord> { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner<RowData> delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner<RowData> delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java new file mode 100644 index 000000000000..35bbb27baf16 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** MapDataStatistics uses map to count key frequency */ +class SketchDataStatistics implements DataStatistics { + + private final ReservoirItemsSketch<SortKey> sketch; + + SketchDataStatistics(int reservoirSize) { + this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); + } + + SketchDataStatistics(ReservoirItemsSketch<SortKey> sketchStats) { + this.sketch = sketchStats; + } + + @Override + public StatisticsType type() { + return StatisticsType.Sketch; + } + + @Override + public boolean isEmpty() { + return sketch.getNumSamples() == 0; + } + + @Override + public void add(SortKey sortKey) { + // clone the sort key first because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + sketch.update(copiedKey); + } + + @Override + public Object result() { + return sketch; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SketchDataStatistics)) { + return false; + } + + ReservoirItemsSketch<SortKey> otherSketch = ((SketchDataStatistics) o).sketch; + return Objects.equal(sketch.getK(), otherSketch.getK()) + && Objects.equal(sketch.getN(), otherSketch.getN()) + && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java new file mode 100644 index 000000000000..dddb0d8722c0 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; + +class SketchRangePartitioner implements Partitioner<RowData> { + private final SortKey sortKey; + private final Comparator<StructLike> comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java new file mode 100644 index 000000000000..871ef9ef1149 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; + +class SketchUtil { + static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; + static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; + static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; + static final int OPERATOR_OVER_SAMPLE_RATIO = 10; + + // switch the statistics tracking from map to sketch if the cardinality of the sort key is over + // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. + static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; + static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; + + private SketchUtil() {} + + /** + * The larger the reservoir size, the more accurate for range bounds calculation and the more + * balanced range distribution. + * + * <p>Here are the heuristic rules + * <li>Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the + * range bounds + * <li>Min is 10K to achieve good accuracy while memory footprint is still relatively small + * <li>Max is 1M to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @return reservoir size + */ + static int determineCoordinatorReservoirSize(int numPartitions) { + int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; + + if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { + // adjust it up and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); + } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { + // adjust it down and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; + } + + return reservoirSize; + } + + /** + * Determine the sampling reservoir size where operator subtasks collect data statistics. + * + * <p>Here are the heuristic rules + * <li>Target size is "coordinator reservoir size * over sampling ration (10) / operator + * parallelism" + * <li>Min is 1K to achieve good accuracy while memory footprint is still relatively small + * <li>Max is 100K to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @param operatorParallelism data statistics operator parallelism + * @return reservoir size + */ + static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { + int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); + int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; + return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); + } + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + * <ul> + * <li>age <= 15 + * <li>age > 15 && age <= 32 + * <li>age >32 && age <= 60 + * <li>age > 60 + * </ul> + * + * <p>Assumption is that a single key is not dominant enough to span multiple subtasks. + * + * @param numPartitions number of partitions which maps to downstream operator parallelism + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + */ + static SortKey[] rangeBounds( + int numPartitions, Comparator<StructLike> comparator, SortKey[] samples) { + // sort the keys first + Arrays.sort(samples, comparator); + int numCandidates = numPartitions - 1; + SortKey[] candidates = new SortKey[numCandidates]; + int step = (int) Math.ceil((double) samples.length / numPartitions); + int position = step - 1; + int numChosen = 0; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; + // skip duplicate values + if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { + // linear probe for the next distinct value + position += 1; + } else { + candidates[numChosen] = candidate; + position += step; + numChosen += 1; + } + } + + return candidates; + } + + /** This can be a bit expensive since it is quadratic. */ + static void convertMapToSketch( + Map<SortKey, Long> taskMapStats, Consumer<SortKey> sketchConsumer) { + taskMapStats.forEach( + (sortKey, count) -> { + for (int i = 0; i < count; ++i) { + sketchConsumer.accept(sortKey); + } + }); + } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator<StructLike> 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/SortKeySerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index d03409f2a430..d1d75019fa2e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -24,7 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; @@ -41,6 +43,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer<SortKey> { @@ -276,13 +279,12 @@ public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot< private Schema schema; private SortOrder sortOrder; - @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { // this constructor is used when restoring from a checkpoint. } - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". @SuppressWarnings("checkstyle:RedundantModifier") public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { this.schema = schema; @@ -320,8 +322,26 @@ public TypeSerializerSchemaCompatibility<SortKey> resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; - return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + // Sort order should be identical + SortKeySerializerSnapshot newSnapshot = + (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); + if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + Set<Integer> sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); + + List<String> compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java new file mode 100644 index 000000000000..d6c23f035015 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link + * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The + * implementation is modeled after {@link ArrayOfStringsSerDe} + */ +class SortKeySketchSerializer extends ArrayOfItemsSerDe<SortKey> implements Serializable { + private static final int DEFAULT_SORT_KEY_SIZE = 128; + + private final TypeSerializer<SortKey> itemSerializer; + private final ListSerializer<SortKey> listSerializer; + private final DataInputDeserializer input; + + SortKeySketchSerializer(TypeSerializer<SortKey> itemSerializer) { + this.itemSerializer = itemSerializer; + this.listSerializer = new ListSerializer<>(itemSerializer); + this.input = new DataInputDeserializer(); + } + + @Override + public byte[] serializeToByteArray(SortKey item) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); + itemSerializer.serialize(item, output); + byte[] itemBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[numBytes + Integer.BYTES]; + ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); + ByteArrayUtil.putIntLE(out, 0, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public byte[] serializeToByteArray(SortKey[] items) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); + listSerializer.serialize(Arrays.asList(items), output); + byte[] itemsBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[Integer.BYTES + numBytes]; + ByteArrayUtil.putIntLE(out, 0, numBytes); + System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return new SortKey[0]; + } + + long offset = startingOffset; + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + offset += Integer.BYTES; + + Util.checkBounds(offset, numBytes, mem.getCapacity()); + byte[] sortKeyBytes = new byte[numBytes]; + mem.getByteArray(offset, sortKeyBytes, 0, numBytes); + input.setBuffer(sortKeyBytes); + + try { + List<SortKey> sortKeys = listSerializer.deserialize(input); + SortKey[] array = new SortKey[numItems]; + sortKeys.toArray(array); + input.releaseArrays(); + return array; + } catch (IOException e) { + throw new UncheckedIOException("Failed to deserialize sort key sketch", e); + } + } + + @Override + public int sizeOf(SortKey item) { + return serializeToByteArray(item).length; + } + + @Override + public int sizeOf(Memory mem, long offset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return 0; + } + + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + return Integer.BYTES + numBytes; + } + + @Override + public String toString(SortKey item) { + return item.toString(); + } + + @Override + public Class<SortKey> getClassOfT() { + return SortKey.class; + } +} 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<SortField> sortFields = sortOrder.fields(); + int size = sortFields.size(); + List<Types.NestedField> 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/sink/shuffle/DataStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 58% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index 852d2157b8cb..f6fcdb8b16ef 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -27,24 +27,39 @@ * statistics in bytes */ @Internal -class DataStatisticsEvent<D extends DataStatistics<D, S>, S> implements OperatorEvent { +class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; + private final boolean applyImmediately; - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; } - static <D extends DataStatistics<D, S>, S> DataStatisticsEvent<D, S> create( + static StatisticsEvent createTaskStatisticsEvent( long checkpointId, - DataStatistics<D, S> dataStatistics, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - return new DataStatisticsEvent<>( + DataStatistics statistics, + TypeSerializer<DataStatistics> statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. + return new StatisticsEvent( checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); + } + + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer<GlobalStatistics> statisticsSerializer, + boolean applyImmediately) { + return new StatisticsEvent( + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); } long checkpointId() { @@ -54,4 +69,8 @@ long checkpointId() { byte[] statisticsBytes() { return statisticsBytes; } + + boolean applyImmediately() { + return applyImmediately; + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 66% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index 889e85112e16..bc28df2b0e22 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.io.Serializable; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.data.RowData; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -34,68 +35,66 @@ * After shuffling, a filter and mapper are required to filter out the data distribution weight, * unwrap the object and extract the original record type T. */ -class DataStatisticsOrRecord<D extends DataStatistics<D, S>, S> implements Serializable { +@Internal +public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private DataStatistics<D, S> statistics; + private GlobalStatistics statistics; private RowData record; - private DataStatisticsOrRecord(DataStatistics<D, S> statistics, RowData record) { + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; this.record = record; } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); + static StatisticsOrRecord fromRecord(RowData record) { + return new StatisticsOrRecord(null, record); } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> fromDataStatistics( - DataStatistics<D, S> statistics) { - return new DataStatisticsOrRecord<>(statistics, null); + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { + return new StatisticsOrRecord(statistics, null); } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> reuseRecord( - DataStatisticsOrRecord<D, S> reuse, TypeSerializer<RowData> recordSerializer) { + static StatisticsOrRecord reuseRecord( + StatisticsOrRecord reuse, TypeSerializer<RowData> recordSerializer) { if (reuse.hasRecord()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> reuseStatistics( - DataStatisticsOrRecord<D, S> reuse, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - if (reuse.hasDataStatistics()) { + static StatisticsOrRecord reuseStatistics( + StatisticsOrRecord reuse, TypeSerializer<GlobalStatistics> statisticsSerializer) { + if (reuse.hasStatistics()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); } } - boolean hasDataStatistics() { + boolean hasStatistics() { return statistics != null; } - boolean hasRecord() { + public boolean hasRecord() { return record != null; } - DataStatistics<D, S> dataStatistics() { + GlobalStatistics statistics() { return statistics; } - void dataStatistics(DataStatistics<D, S> newStatistics) { + void statistics(GlobalStatistics newStatistics) { this.statistics = newStatistics; } - RowData record() { + public RowData record() { return record; } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 53% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index e9a6fa0cbfc5..6e403425938d 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -29,13 +29,12 @@ import org.apache.flink.table.data.RowData; @Internal -class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S> - extends TypeSerializer<DataStatisticsOrRecord<D, S>> { - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; +class StatisticsOrRecordSerializer extends TypeSerializer<StatisticsOrRecord> { + private final TypeSerializer<GlobalStatistics> statisticsSerializer; private final TypeSerializer<RowData> recordSerializer; - DataStatisticsOrRecordSerializer( - TypeSerializer<DataStatistics<D, S>> statisticsSerializer, + StatisticsOrRecordSerializer( + TypeSerializer<GlobalStatistics> statisticsSerializer, TypeSerializer<RowData> recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,13 +47,13 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() { - TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer = + public TypeSerializer<StatisticsOrRecord> duplicate() { + TypeSerializer<GlobalStatistics> duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( + return new StatisticsOrRecordSerializer( duplicateStatisticsSerializer, duplicateRowDataSerializer); } else { return this; @@ -62,34 +61,31 @@ public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() { } @Override - public DataStatisticsOrRecord<D, S> createInstance() { + public StatisticsOrRecord createInstance() { // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } @Override - public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) { + public StatisticsOrRecord copy(StatisticsOrRecord from) { if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); } } @Override - public DataStatisticsOrRecord<D, S> copy( - DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) { - DataStatisticsOrRecord<D, S> to; + public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { + StatisticsOrRecord to; if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.copy(from.record(), to.record()); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics<D, S> statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); + to.statistics(statistics); } return to; @@ -101,41 +97,40 @@ public int getLength() { } @Override - public void serialize(DataStatisticsOrRecord<D, S> statisticsOrRecord, DataOutputView target) + public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) throws IOException { if (statisticsOrRecord.hasRecord()) { target.writeBoolean(true); recordSerializer.serialize(statisticsOrRecord.record(), target); } else { target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); } } @Override - public DataStatisticsOrRecord<D, S> deserialize(DataInputView source) throws IOException { + public StatisticsOrRecord deserialize(DataInputView source) throws IOException { boolean isRecord = source.readBoolean(); if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); } } @Override - public DataStatisticsOrRecord<D, S> deserialize( - DataStatisticsOrRecord<D, S> reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord<D, S> to; + public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) + throws IOException { + StatisticsOrRecord to; boolean isRecord = source.readBoolean(); if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.deserialize(to.record(), source); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics<D, S> statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + to.statistics(statistics); } return to; @@ -154,12 +149,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + if (!(obj instanceof StatisticsOrRecordSerializer)) { return false; } - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer<D, S> other = (DataStatisticsOrRecordSerializer<D, S>) obj; + StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; return Objects.equals(statisticsSerializer, other.statisticsSerializer) && Objects.equals(recordSerializer, other.recordSerializer); } @@ -170,25 +164,22 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot<DataStatisticsOrRecord<D, S>> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); + public TypeSerializerSnapshot<StatisticsOrRecord> snapshotConfiguration() { + return new StatisticsOrRecordSerializerSnapshot(this); } - public static class DataStatisticsOrRecordSerializerSnapshot<D extends DataStatistics<D, S>, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord<D, S>, DataStatisticsOrRecordSerializer<D, S>> { + public static class StatisticsOrRecordSerializerSnapshot + extends CompositeTypeSerializerSnapshot<StatisticsOrRecord, StatisticsOrRecordSerializer> { private static final int CURRENT_VERSION = 1; - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public StatisticsOrRecordSerializerSnapshot() { + super(StatisticsOrRecordSerializer.class); } @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer<D, S> serializer) { + public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { super(serializer); } @@ -200,7 +191,7 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer<?>[] getNestedSerializers( - DataStatisticsOrRecordSerializer<D, S> outerSerializer) { + StatisticsOrRecordSerializer outerSerializer) { return new TypeSerializer<?>[] { outerSerializer.statisticsSerializer, outerSerializer.recordSerializer }; @@ -208,12 +199,12 @@ protected TypeSerializer<?>[] getNestedSerializers( @SuppressWarnings("unchecked") @Override - protected DataStatisticsOrRecordSerializer<D, S> createOuterSerializerWithNestedSerializers( + protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer<?>[] nestedSerializers) { - TypeSerializer<DataStatistics<D, S>> statisticsSerializer = - (TypeSerializer<DataStatistics<D, S>>) nestedSerializers[0]; + TypeSerializer<GlobalStatistics> statisticsSerializer = + (TypeSerializer<GlobalStatistics>) nestedSerializers[0]; TypeSerializer<RowData> recordSerializer = (TypeSerializer<RowData>) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java new file mode 100644 index 000000000000..43f72e336e06 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +/** + * Range distribution requires gathering statistics on the sort keys to determine proper range + * boundaries to distribute/cluster rows before writer operators. + */ +public enum StatisticsType { + /** + * Tracks the data statistics as {@code Map<SortKey, Long>} frequency. It works better for + * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in + * hundreds or thousands. + * + * <ul> + * <li>Pro: accurate measurement on the statistics/weight of every key. + * <li>Con: memory footprint can be large if the key cardinality is high. + * </ul> + */ + Map, + + /** + * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds + * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, + * uuid etc.) where the cardinalities can be in millions or billions. + * + * <ul> + * <li>Pro: relatively low memory footprint for high-cardinality sort keys. + * <li>Con: non-precise approximation with potentially lower accuracy. + * </ul> + */ + Sketch, + + /** + * Initially use Map for statistics tracking. If key cardinality turns out to be high, + * automatically switch to sketch sampling. + */ + Auto +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java new file mode 100644 index 000000000000..5d48ec57ca49 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.io.UncheckedIOException; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class StatisticsUtil { + + private StatisticsUtil() {} + + static DataStatistics createTaskStatistics( + StatisticsType type, int operatorParallelism, int numPartitions) { + if (type == StatisticsType.Map) { + return new MapDataStatistics(); + } else { + return new SketchDataStatistics( + SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); + } + } + + static byte[] serializeDataStatistics( + DataStatistics dataStatistics, TypeSerializer<DataStatistics> statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize data statistics", e); + } + } + + static DataStatistics deserializeDataStatistics( + byte[] bytes, TypeSerializer<DataStatistics> statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize data statistics", e); + } + } + + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer<CompletedStatistics> statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(completedStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer<CompletedStatistics> statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer<GlobalStatistics> statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer<GlobalStatistics> statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static StatisticsType collectType(StatisticsType config) { + return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; + } + + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + + static StatisticsType collectType( + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } +} diff --git a/flink/v1.18/flink/src/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..5718f4b93825 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -28,6 +28,8 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +39,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -61,10 +66,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -72,6 +79,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; @@ -95,6 +103,11 @@ public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEn private final SerializableRecordEmitter<T> emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List<IcebergSourceSplit> batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -130,16 +143,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List<IcebergSourceSplit> planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List<IcebergSourceSplit> splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -205,26 +228,69 @@ private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumer // Only do scan planning if nothing is restored from checkpoint state List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static <T> Builder<T> builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder<RowData> forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param <T> output type + * @return an IcebergSource builder + */ + public static <T> Builder<T> forOutputType(RowDataConverter<T> converter) { + return new Builder<T>().converter(converter); + } + public static class Builder<T> { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator<IcebergSourceSplit> splitComparator; private ReaderFunction<T> readerFunction; + private RowDataConverter<T> converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +321,28 @@ public Builder<T> splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder<T> readerFunction(ReaderFunction<T> newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder<T> converter(RowDataConverter<T> newConverter) { + this.converter = newConverter; + return this; + } + public Builder<T> flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -482,6 +565,10 @@ public IcebergSource<T> build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); @@ -506,24 +593,7 @@ public IcebergSource<T> build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters()); - this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -544,5 +614,75 @@ public IcebergSource<T> build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream<T> buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource<T> source = build(); + TypeInformation<T> outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource<T> stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static <T> TypeInformation<T> outputTypeInfo( + RowDataConverter<T> converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation<T>) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + + private ReaderFunction<T> readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction<T>) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction<T>) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource<RowData> createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream<RowData> createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource<RowData> source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.18/flink/src/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<IcebergSourceSplit, IcebergEnumeratorState> { + implements SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState>, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext<IcebergSourceSplit> 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<IcebergSourceSplit> 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/AvroGenericRecordConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter<GenericRecord> { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation<GenericRecord> outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation<GenericRecord> getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction<GenericRecord> { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction<T> extends DataIteratorReaderFunction<T> { + private final RowDataConverter<T> converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List<Expression> filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter<T> converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List<Expression> filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator<T> createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader<T> implements FileScanTaskReader<T> { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter<T> converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter<T> converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator<T> open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/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<T> extends DataIterator<T> { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader<T> 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/RowDataConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param <T> output type + */ +public interface RowDataConverter<T> + extends Function<RowData, T>, ResultTypeQueryable<T>, Serializable {} 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<RowData> { private final FileIO io; private final EncryptionManager encryption; private final List<Expression> 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<Expression> 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<Expression> 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<RowData> 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/CatalogTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java index 91ed3c4adea3..062ff68d5d85 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -118,26 +118,4 @@ protected String getFullQualifiedTableName(String tableName) { static String getURI(HiveConf conf) { return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); } - - static String toWithClause(Map<String, String> props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry<String, String> entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/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/MiniFlinkClusterExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java index 9a73b80e077d..d2e086aa448e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; @@ -50,4 +51,17 @@ public static MiniClusterExtension createWithClassloaderCheckDisabled() { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ce6caca12158..1767f774922a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Duration; @@ -71,7 +72,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; -import org.junit.Assert; public class SimpleDataUtil { @@ -268,13 +268,13 @@ public static boolean equalsRecords(List<Record> expected, List<Record> actual, } public static void assertRecordsEqual(List<Record> expected, List<Record> actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); + assertThat(actual).hasSameSizeAs(expected); Types.StructType type = schema.asStruct(); StructLikeSet expectedSet = StructLikeSet.create(type); expectedSet.addAll(expected); StructLikeSet actualSet = StructLikeSet.create(type); actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } /** @@ -297,7 +297,7 @@ public static void assertTableRecords(Table table, List<Record> expected, String Snapshot snapshot = latestSnapshot(table, branch); if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); + assertThat(expected).isEmpty(); return; } @@ -313,7 +313,7 @@ public static void assertTableRecords(Table table, List<Record> expected, String actualSet.add(record); } - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List<Row> sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator<Row> iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable<Row> expected, Iterable<Row> actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable<Row> expected, Iterable<Row> 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<String, String> props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry<String, String> entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..401960c3591b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink; +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; import static org.assertj.core.api.Assertions.assertThat; import java.nio.file.Path; @@ -26,7 +27,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -40,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<Row> expected, Iterab * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog */ protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java index 884ea2d1d3b1..b9a7d5b1d589 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -46,6 +46,8 @@ private TestFixtures() {} public static final String SINK_TABLE = "t_sink"; public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); public static final Schema TS_SCHEMA = new Schema( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..2978a92945a2 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -18,36 +18,21 @@ */ package org.apache.iceberg.flink; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -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<Row> 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<String, String> 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<Row> 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<Long, List<DataFile>> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List<DataFile> dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java new file mode 100644 index 000000000000..b63547d433a4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List<Object[]> 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<String, String> 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<Row> 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<Row> 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<String, String> 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<Long, List<DataFile>> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List<DataFile> dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List<List<Row>> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List<Row> charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List<Row> flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map<String, String> tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/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..47f5485df879 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -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<String, String> props) { - return CatalogTestBase.toWithClause(props); - } - private String createWarehouse() { try { return String.format( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index 7dd4e8759c0e..e76452b7cea0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -39,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -212,7 +211,6 @@ public void testBasicProjection() throws Exception { projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); assertThat(projected.getString(0)).asString().isEqualTo("test"); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/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/TestCompressionSettings.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 14f12422da96..3299e7a97776 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; @@ -33,55 +40,50 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; - private final Map<String, String> initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } + @Parameter(index = 0) + private Map<String, String> initProperties; - public TestCompressionSettings(Map<String, String> initProperties) { - this.initProperties = initProperties; + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; } - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); } - @Test + @TestTemplate public void testCompressionAvro() throws Exception { // No override provided Map<String, String> resultProperties = @@ -91,19 +93,17 @@ public void testCompressionAvro() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -119,11 +119,12 @@ public void testCompressionAvro() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionParquet() throws Exception { // No override provided Map<String, String> resultProperties = @@ -133,19 +134,19 @@ public void testCompressionParquet() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -161,11 +162,12 @@ public void testCompressionParquet() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionOrc() throws Exception { // No override provided Map<String, String> resultProperties = @@ -175,19 +177,18 @@ public void testCompressionOrc() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); } // Override compression to snappy and a different strategy @@ -203,12 +204,15 @@ public void testCompressionOrc() throws Exception { FlinkWriteOptions.COMPRESSION_STRATEGY.key(), "speed")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map<String, String> override) throws Exception { + private static OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map<String, String> override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -216,7 +220,7 @@ private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIce IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness<RowData, WriteResult> harness = + OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -227,7 +231,7 @@ private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIce private static Map<String, String> appenderProperties( Table table, TableSchema schema, Map<String, String> override) throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..b778037c559c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,20 +18,11 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - import java.io.IOException; -import java.util.Collections; import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -39,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<Row> rows = createRows(""); - DataStream<Row> 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<String, String> 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<Row> leftRows = createRows("left-"); - DataStream<Row> 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<Row> rightRows = createRows("right-"); - DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List<Row> rows = createRows(""); - DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List<Row> rows = createRows(""); - DataStream<Row> 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<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream<RowData> dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index b38aa6b50ce6..9ce36cc1e8d0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -18,29 +18,52 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkIcebergSinkBase { - protected Table table; - protected StreamExecutionEnvironment env; + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + protected static final TypeInformation<Row> 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<Row> createBoundedSource(List<Row> rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List<Row> createRows(String prefix) { protected List<RowData> convertToRowData(List<Row> rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/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/TestFlinkIcebergSinkDistributionMode.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java new file mode 100644 index 000000000000..df8c3c79d3e3 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,494 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +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<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> 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 testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + + @TestTemplate + public void testRangeDistributionWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List<Row> rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream<Row> dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + // sometimes + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource<Row> createRangeDistributionBoundedSource( + List<List<Row>> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List<List<Row>> createCharRows(int numOfCheckpoints, int countPerChar) { + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List<Row> rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List<List<Row>> createIntRows(int numOfCheckpoints, int maxId) { + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List<Row> rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map<String, String> 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<Row> leftRows = createRows("left-"); + DataStream<Row> 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<Row> rightRows = createRows("right-"); + DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> 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<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream<RowData> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..b283b8390a2b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -38,7 +39,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 +57,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 +89,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -185,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.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/TestFlinkManifest.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index ce1f208a4b07..53b7c4c0cc91 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -19,9 +19,12 @@ package org.apache.iceberg.flink.sink; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -45,28 +48,26 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkManifest { private static final Configuration CONF = new Configuration(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; private FileAppenderFactory<RowData> appenderFactory; private final AtomicInteger fileCount = new AtomicInteger(0); - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + assertThat(new File(tablePath).mkdir()).isTrue(); // Construct the iceberg table. table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); @@ -112,11 +113,11 @@ public void testIO() throws IOException { WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < 5; i++) { TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); } @@ -131,7 +132,7 @@ public void testUserProvidedManifestLocation() throws IOException { long checkpointId = 1; String flinkJobId = newFlinkJobId(); String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map<String, String> props = ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = @@ -144,21 +145,18 @@ public void testUserProvidedManifestLocation() throws IOException { () -> factory.create(checkpointId), table.spec()); - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } @@ -198,7 +196,7 @@ public void testVersionedSerializer() throws IOException { byte[] versionedSerializeData2 = SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); } @Test @@ -220,14 +218,13 @@ public void testCompatibility() throws IOException { DeltaManifests delta = SimpleVersionedSerialization.readVersionAndDeSerialize( DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); TestHelpers.assertEquals(manifest, delta.dataManifest()); List<DataFile> actualFiles = FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); + assertThat(actualFiles).hasSize(10); for (int i = 0; i < 10; i++) { TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 948c7b31430c..ac5babe11943 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -161,7 +162,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +178,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +195,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +207,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +236,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +247,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +290,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +301,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +342,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +355,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +368,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +382,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +407,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List<RowData> expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +419,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +455,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +482,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +505,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +519,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +537,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +551,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +576,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +589,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +613,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness<WriteResult, Void> harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +632,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +655,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness<WriteResult, Void> harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +682,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +708,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +721,14 @@ public void testBoundedStream() throws Exception { List<RowData> tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +741,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +752,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +792,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +802,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +834,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +866,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +881,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +895,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +912,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + * <ul> + * <li>A specific row is updated + * <li>The prepareSnapshotPreBarrier triggered + * <li>Checkpoint failed for reasons outside of the Iceberg connector + * <li>The specific row is updated again in the second checkpoint as well + * <li>Second snapshot is triggered, and finished + * </ul> + * + * <p>Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory<RowData> appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +997,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1009,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1028,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1046,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1063,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1089,7 +1189,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness<WriteResult, Void> createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1209,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory<Void> - implements OneInputStreamOperatorFactory<WriteResult, Void> { + implements OneInputStreamOperatorFactory<FlinkWriteResult, Void> { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index c05f08b82f51..e13721a9f170 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -22,11 +22,13 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -42,6 +44,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -58,52 +63,47 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; private Table table; - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table. Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,10 +111,11 @@ public void testWritingTable() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); checkpointId = checkpointId + 1; @@ -124,9 +125,10 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); @@ -145,49 +147,53 @@ public void testWritingTable() throws Exception { } } - @Test + @TestTemplate public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } } - @Test + @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); + assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); + assertThat(scanDataFiles()).hasSize(1); } private Set<String> scanDataFiles() throws IOException { @@ -211,9 +217,9 @@ private Set<String> scanDataFiles() throws IOException { } } - @Test + @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,45 +227,49 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); testHarness.endInput(); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testTableWithTargetFileSize() throws Exception { // Adjust the target-file-size in table properties. table @@ -276,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -284,13 +294,14 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); // Assert that the data file have the expected records. for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); + assertThat(dataFile.recordCount()).isEqualTo(1000); } // Commit the iceberg transaction. @@ -303,7 +314,7 @@ public void testTableWithTargetFileSize() throws Exception { SimpleDataUtil.assertTableRecords(table, records); } - @Test + @TestTemplate public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( @@ -329,7 +340,8 @@ public void testPromotedFlinkDataType() throws Exception { spec = PartitionSpec.unpartitioned(); } - String location = tempFolder.newFolder().getAbsolutePath(); + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); @@ -346,15 +358,16 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter() + private static List<WriteResult> getWriteResults(List<FlinkWriteResult> flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStr IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness<RowData, WriteResult> harness = + OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java index 79be96d20902..919fef579ab0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +37,7 @@ import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRowDataPartitionKey { private static final Schema SCHEMA = @@ -91,10 +92,10 @@ public void testNullPartitionValue() { for (RowData row : rows) { PartitionKey partitionKey = new PartitionKey(spec, schema); partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); + assertThat(partitionKey.size()).isEqualTo(1); String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); } } @@ -116,15 +117,15 @@ public void testPartitionWithOneNestedField() { PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); + assertThat(partitionKey1.size()).isEqualTo(1); - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); + assertThat(partitionKey2.size()).isEqualTo(1); - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); } } @@ -154,16 +155,16 @@ public void testPartitionMultipleNestedField() { Record record = (Record) records.get(i).get(0); pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); + assertThat(pk1.size()).isEqualTo(2); - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); + assertThat(pk2.size()).isEqualTo(2); - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); } } @@ -190,19 +191,18 @@ public void testPartitionValueTypes() { pk.partition(rowWrapper.wrap(row)); expectedPK.partition(recordWrapper.wrap(record)); - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } @@ -232,19 +232,18 @@ public void testNestedPartitionValues() { pk.partition(rowWrapper.wrap(rows.get(j))); expectedPK.partition(recordWrapper.wrap(records.get(j))); - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 112dbb511310..8bfd6cb3d043 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; @@ -30,6 +33,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -39,69 +45,62 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestTaskWriters { private static final Configuration CONF = new Configuration(); private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; - private Table table; + @Parameter(index = 1) + private boolean partitioned; - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } + private Table table; - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table with the specified file format. Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); // Close again. taskWriter.close(); dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); } } - @Test + @TestTemplate public void testCloseTwice() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -111,16 +110,16 @@ public void testCloseTwice() throws IOException { int expectedFiles = partitioned ? 2 : 1; DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } } } - @Test + @TestTemplate public void testAbort() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -130,16 +129,16 @@ public void testAbort() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); } } } - @Test + @TestTemplate public void testCompleteFiles() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "a")); @@ -149,14 +148,14 @@ public void testCompleteFiles() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); @@ -176,7 +175,7 @@ public void testCompleteFiles() throws IOException { } } - @Test + @TestTemplate public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(4)) { List<RowData> rows = Lists.newArrayListWithCapacity(8000); @@ -193,7 +192,7 @@ public void testRollingWithTargetFileSize() throws IOException { } DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); + assertThat(dataFiles).hasSize(8); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { @@ -206,7 +205,7 @@ public void testRollingWithTargetFileSize() throws IOException { } } - @Test + @TestTemplate public void testRandomData() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable<RowData> rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java new file mode 100644 index 000000000000..5910bd685510 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +class Fixtures { + private Fixtures() {} + + public static final int NUM_SUBTASKS = 2; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); + public static final TypeSerializer<RowData> ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); + public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + public static final Comparator<StructLike> SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); + public static final SortKeySerializer SORT_KEY_SERIALIZER = + new SortKeySerializer(SCHEMA, SORT_ORDER); + public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = + new DataStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); + + public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + public static final Map<String, SortKey> CHAR_KEYS = createCharKeys(); + + public static StatisticsEvent createStatisticsEvent( + StatisticsType type, + TypeSerializer<DataStatistics> statisticsSerializer, + long checkpointId, + SortKey... keys) { + DataStatistics statistics = createTaskStatistics(type, keys); + return StatisticsEvent.createTaskStatisticsEvent( + checkpointId, statistics, statisticsSerializer); + } + + public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { + DataStatistics statistics; + if (type == StatisticsType.Sketch) { + statistics = new SketchDataStatistics(128); + } else { + statistics = new MapDataStatistics(); + } + + for (SortKey key : keys) { + statistics.add(key); + } + + return statistics; + } + + private static Map<String, SortKey> createCharKeys() { + Map<String, SortKey> keys = Maps.newHashMap(); + for (char c = 'a'; c <= 'z'; ++c) { + String key = Character.toString(c); + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, key); + keys.put(key, sortKey); + } + + return keys; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index 890cc361b246..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Test; - -public class TestAggregatedStatistics { - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - @Test - public void mergeDataStatisticTest() { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - - AggregatedStatistics<MapDataStatistics, Map<SortKey, Long>> aggregatedStatistics = - new AggregatedStatistics<>(1, statisticsSerializer); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(keyA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyB) - + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 4c64ce522201..8322ce683768 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -18,161 +18,448 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - private final SortKey keyA = sortKey.copy(); - private final SortKey keyB = sortKey.copy(); - - private AggregatedStatisticsTracker<MapDataStatistics, Map<SortKey, Long>> - aggregatedStatisticsTracker; - - public TestAggregatedStatisticsTracker() { - keyA.set(0, "a"); - keyB.set(0, "b"); - } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveNewerStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); - @Before - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + // checkpoint 1 is completed + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + // checkpoint 2 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } } - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - checkpoint2Subtask0DataStatistic.add(keyB); - checkpoint2Subtask0DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveOlderStatisticsEventTest(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint3Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); + // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 4L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + } } - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveCompletedStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics<MapDataStatistics, Map<SortKey, Long>> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(keyA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyA) - + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); - assertThat((long) globalDataStatistics.statistics().get(keyB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyB) - + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(keyB); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 3L, + CHAR_KEYS.get("b"), 3L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint2Subtask0DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); + completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask1DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); // Receive data statistics from all subtasks at checkpoint 2 completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); + tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + } + + @Test + public void coordinatorSwitchToSketchOverThreshold() { + int parallelism = 3; + int downstreamParallelism = 3; + int switchToSketchThreshold = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + switchToSketchThreshold, + null); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + // converted to sketch statistics as map size is 4 (over the switch threshold of 3) + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + @Test + public void coordinatorMapOperatorSketch() { + int parallelism = 3; + int downstreamParallelism = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + + // first operator event has map statistics + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + // second operator event contains sketch statistics + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Sketch, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + // third operator event has Map statistics + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + private AggregatedStatisticsTracker createTracker(StatisticsType type) { + return new AggregatedStatisticsTracker( + "testOperator", + Fixtures.NUM_SUBTASKS, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java similarity index 70% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java index 0ce73fa4aaad..4ee9888934a8 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -25,11 +25,11 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -public class TestAggregatedStatisticsSerializer extends SerializerTestBase<AggregatedStatistics> { +public class TestCompletedStatisticsSerializer extends SerializerTestBase<CompletedStatistics> { @Override - protected TypeSerializer<AggregatedStatistics> createSerializer() { - return Fixtures.AGGREGATED_STATISTICS_SERIALIZER; + protected TypeSerializer<CompletedStatistics> createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; } @Override @@ -38,17 +38,17 @@ protected int getLength() { } @Override - protected Class<AggregatedStatistics> getTypeClass() { - return AggregatedStatistics.class; + protected Class<CompletedStatistics> getTypeClass() { + return CompletedStatistics.class; } @Override - protected AggregatedStatistics[] getTestData() { - return new AggregatedStatistics[] { - AggregatedStatistics.fromKeyFrequency( + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), - AggregatedStatistics.fromRangeBounds( - 2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) }; } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 3df714059c37..a08a8a73e80c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -18,9 +18,13 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.time.Duration; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -28,128 +32,182 @@ import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> - dataStatisticsCoordinator; - @Before + @BeforeEach public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); } - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + private void tasksReady(DataStatisticsCoordinator coordinator) { + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); } - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + StatisticsEvent.createTaskStatisticsEvent( + 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + Map<SortKey, Long> keyFrequency = + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 3L, + CHAR_KEYS.get("c"), 5L); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(NUM_SUBTASKS, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.checkpointId()).isEqualTo(1L); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } } @Test - public void testDataStatisticsEventHandling() throws Exception { - tasksReady(); - SortKey key = sortKey.copy(); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify global data statistics is the aggregation of all subtasks data statistics - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - keyA, 2L, - keyB, 3L, - keyC, 5L)); + public void testRequestGlobalStatisticsEventHandling() throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); + } } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> dataStatisticsCoordinator, + DataStatisticsCoordinator dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -157,8 +215,7 @@ static void setAllTasksReady( } } - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> coordinator) { + static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { CompletableFuture<Void> future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { @@ -175,4 +232,15 @@ static void waitForCoordinatorToProcessActions( ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); } } + + private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { + return new DataStatisticsCoordinator( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + NUM_SUBTASKS, + type, + 0.0d); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index 5e0a752be506..6317f2bfde18 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; @@ -27,117 +31,157 @@ import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsCoordinatorProvider<MapDataStatistics, Map<SortKey, Long>> provider; private EventReceivingTasks receivingTasks; - @Before + @BeforeEach public void before() { - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); receivingTasks = EventReceivingTasks.createForRunningTasks(); } - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - SortKey keyD = sortKey.copy(); - keyD.set(0, "c"); - SortKey keyE = sortKey.copy(); - keyE.set(0, "c"); - + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testCheckpointAndReset(StatisticsType type) throws Exception { + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> dataStatisticsCoordinator = - (DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>>) - coordinator.getInternalCoordinator(); + provider.create( + new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { + DataStatisticsCoordinator dataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); // Start the coordinator coordinator.start(); TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyC); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + Map<SortKey, Long> checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyD); - checkpoint2Subtask0DataStatistic.add(keyE); - checkpoint2Subtask0DataStatistic.add(keyE); - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); + coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + Map<SortKey, Long> checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + } + waitForCheckpoint(2L, dataStatisticsCoordinator); // Reset coordinator to checkpoint 1 coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + DataStatisticsCoordinator restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } } } - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator<MapDataStatistics, Map<SortKey, Long>> coordinator) + private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) throws InterruptedException, ExecutionException { CompletableFuture<byte[]> future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); return future.get(); } + + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { + return new DataStatisticsCoordinatorProvider( + "DataStatisticsCoordinatorProvider", + OPERATOR_ID, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + type, + 0.0); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 0e99a2d74ccb..c760f1ba96d3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -18,22 +18,25 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.StateInitializationContext; @@ -49,102 +52,95 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; public class TestDataStatisticsOperator { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final RowType rowType = RowType.of(new VarCharType(), new IntType()); - private final TypeSerializer<RowData> rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer<DataStatistics<MapDataStatistics, Map<SortKey, Long>>> - statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer( - new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } - @Before + private Environment env; + + @BeforeEach public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask<String, String>(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); + this.env = + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); } - private DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> createOperator() { + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return new DataStatisticsOperator<>( - "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); + return createOperator(type, downstreamParallelism, mockGateway); } - @After - public void clean() throws Exception { - operator.close(); + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { + DataStatisticsOperator operator = + new DataStatisticsOperator( + "testOperator", + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + mockGateway, + downstreamParallelism, + type); + operator.setup( + new OneInputStreamTask<String, String>(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + return operator; } - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness = createHarness(this.operator)) { + @SuppressWarnings("unchecked") + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElement(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - Map<SortKey, Long> expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); - - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map<SortKey, Long> statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + DataStatistics localStatistics = operator.localStatistics(); + assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + Map<SortKey, Long> keyFrequency = (Map<SortKey, Long>) localStatistics.result(); + assertThat(keyFrequency) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); + } else { + ReservoirItemsSketch<SortKey> sketch = + (ReservoirItemsSketch<SortKey>) localStatistics.result(); + assertThat(sketch.getSamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } testHarness.endInput(); } } - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness = createHarness(this.operator)) { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testOperatorOutput(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { testHarness.processElement( new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); testHarness.processElement( @@ -154,8 +150,8 @@ public void testOperatorOutput() throws Exception { List<RowData> recordsOutput = testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record) .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( @@ -166,70 +162,172 @@ public void testOperatorOutput() throws Exception { } } - @Test - public void testRestoreState() throws Exception { + private static Stream<Arguments> provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + + @ParameterizedTest + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { + Map<SortKey, Long> keyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); + SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness1 = createHarness(this.operator)) { - MapDataStatistics mapDataStatistics = new MapDataStatistics(); - - SortKey key = sortKey.copy(); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "b"); - mapDataStatistics.add(key); - key.set(0, "c"); - mapDataStatistics.add(key); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map<SortKey, Long> expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - DataStatisticsEvent<MapDataStatistics, Map<SortKey, Long>> event = - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness1 = + createHarness(operator)) { + GlobalStatistics statistics; + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); + } else { + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); + } + + StatisticsEvent event = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); operator.handleOperatorEvent(event); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf(expectedMap); + + GlobalStatistics globalStatistics = operator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - // restored RowData is BinaryRowData. convert to GenericRowData for comparison - Map<SortKey, Long> restoredStatistics = Maps.newHashMap(); - restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + } + } - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map<SortKey, Long> expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithLocalStatsOverThreshold() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // add rows with unique keys + for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map<SortKey, Long>) operator.localStatistics().result()).hasSize(i + 1); + } + + // one more item should trigger the migration to sketch stats + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch<SortKey> sketch = + (ReservoirItemsSketch<SortKey>) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); + // reservoir not full yet + assertThat(sketch.getN()).isLessThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); + + // add more items to saturate the reservoir + for (int i = 0; i < reservoirSize; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + } + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + sketch = (ReservoirItemsSketch<SortKey>) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()) + .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); + // reservoir is full now + assertThat(sketch.getN()).isGreaterThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize(reservoirSize); + + testHarness.endInput(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithGlobalSketchStatistics() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); - assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); + // started with Map stype + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map<SortKey, Long>) operator.localStatistics().result()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + + // received global statistics with sketch type + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( + 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); + operator.handleOperatorEvent( + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch<SortKey> sketch = + (ReservoirItemsSketch<SortKey>) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(1); + assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); + + testHarness.endInput(); } } private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); OperatorStateStore operatorStateStore = @@ -238,17 +336,14 @@ private StateInitializationContext getStateContext() throws Exception { return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); } - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - createHarness( - final DataStatisticsOperator<MapDataStatistics, Map<SortKey, Long>> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord<MapDataStatistics, Map<SortKey, Long>>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); + private OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> createHarness( + DataStatisticsOperator dataStatisticsOperator) throws Exception { + OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> harness = + new OneInputStreamOperatorTestHarness<>( + dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); + harness.setup( + new StatisticsOrRecordSerializer( + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); harness.open(); return harness; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java new file mode 100644 index 000000000000..59ce6df05d9d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class TestDataStatisticsSerializer extends SerializerTestBase<DataStatistics> { + @Override + protected TypeSerializer<DataStatistics> createSerializer() { + return Fixtures.TASK_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<DataStatistics> getTypeClass() { + return DataStatistics.class; + } + + @Override + protected DataStatistics[] getTestData() { + return new DataStatistics[] { + new MapDataStatistics(), + Fixtures.createTaskStatistics( + StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), + new SketchDataStatistics(128), + Fixtures.createTaskStatistics( + StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) + }; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase<GlobalStatistics> { + + @Override + protected TypeSerializer<GlobalStatistics> createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<GlobalStatistics> getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index be2beeebc93c..8a25c7ad9898 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -18,74 +18,50 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { - private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); - private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private final RowDataWrapper rowWrapper = - new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); - + @SuppressWarnings("unchecked") @Test public void testAddsAndGet() { MapDataStatistics dataStatistics = new MapDataStatistics(); - GenericRowData reusedRow = - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("c")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("a")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - Map<SortKey, Long> actual = dataStatistics.statistics(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyA = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyB = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyC = sortKey.copy(); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); - Map<SortKey, Long> expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Map<SortKey, Long> actual = (Map<SortKey, Long>) dataStatistics.result(); + Map<SortKey, Long> expected = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index e6726e7db785..d5a0bebc74e7 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -64,65 +65,60 @@ private static SortKey[] initSortKeys() { } // Total weight is 800 - private final MapDataStatistics mapDataStatistics = - new MapDataStatistics( - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L)); + private final Map<SortKey, Long> mapStatistics = + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L); @Test public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // each task should get targeted weight of 100 (=800/8) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -144,19 +140,20 @@ public void testEvenlyDividableNoClosingFileCost() { Pair.of(100L, 1), 7, Pair.of(100L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 100 before close file cost factored in. // close file cost is 5 = 5% * 100. @@ -165,35 +162,30 @@ public void testEvenlyDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight with close cost per subtask is 110 (880/8) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -216,51 +208,48 @@ public void testEvenlyDividableWithClosingFileCost() { Pair.of(100L, 2), 7, Pair.of(75L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 // each task should get targeted weight of 89 = ceiling(800/9) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -284,19 +273,20 @@ public void testNonDividableNoClosingFileCost() { Pair.of(89L, 2), 8, Pair.of(88L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 89 before close file cost factored in. // close file cost is 5 (= 5% * 89) per file. @@ -305,35 +295,31 @@ public void testNonDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight per subtask is 98 ceiling(880/9) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -358,40 +344,39 @@ public void testNonDividableWithClosingFileCost() { Pair.of(88L, 2), 8, Pair.of(61L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); // drift threshold is high for non-dividable scenario with close cost validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); } private static Map<Integer, Pair<AtomicLong, Set<RowData>>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { + MapRangePartitioner partitioner, int numPartitions, Map<SortKey, Long> mapStatistics) { // The Map key is the subtaskId. // For the map value pair, the first element is the count of assigned and // the second element of Set<String> is for the set of assigned keys. Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair<AtomicLong, Set<RowData>> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair<AtomicLong, Set<RowData>> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); return partitionResults; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/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<Integer> 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<Integer> 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/TestSketchDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java new file mode 100644 index 000000000000..396bfae2f13c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + SketchDataStatistics dataStatistics = new SketchDataStatistics(128); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + ReservoirItemsSketch<SortKey> actual = (ReservoirItemsSketch<SortKey>) dataStatistics.result(); + assertThat(actual.getSamples()) + .isEqualTo( + new SortKey[] { + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") + }); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/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 + * + * <ul> + * <li>age <= 15 + * <li>age > 15 && age <= 32 + * <li>age >32 && age <= 60 + * <li>age > 60 + * </ul> + */ + 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 new file mode 100644 index 000000000000..16202c075ea0 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSketchUtil { + @Test + public void testCoordinatorReservoirSize() { + // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); + // adjust to multiplier of 100 + assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); + // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) + .isEqualTo(1_000_000 - (1_000_000 % 10_123)); + } + + @Test + public void testOperatorReservoirSize() { + assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) + .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); + assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) + .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); + assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) + .isEqualTo( + (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); + assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) + .isEqualTo( + (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); + } + + @Test + public void testRangeBoundsOneChannel() { + assertThat( + SketchUtil.rangeBounds( + 1, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .isEmpty(); + } + + @Test + public void testRangeBoundsDivisible() { + assertThat( + SketchUtil.rangeBounds( + 3, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void testRangeBoundsNonDivisible() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); + } + + @Test + public void testRangeBoundsSkipDuplicates() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + // skipped duplicate c's + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); + } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java index 291302aef486..54cceae6e55b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -18,14 +18,24 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.iceberg.NullOrder; import org.apache.iceberg.Schema; import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.RowDataWrapper; +import org.junit.jupiter.api.Test; public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { private final DataGenerator generator = new DataGenerators.Primitives(); @@ -54,4 +64,27 @@ protected SortOrder sortOrder() { protected GenericRowData rowData() { return generator.generateFlinkRowData(); } + + @Test + public void testSerializationSize() throws Exception { + RowData rowData = + GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = Fixtures.SORT_KEY.copy(); + sortKey.wrap(struct); + SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataOutputSerializer output = new DataOutputSerializer(1024); + serializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + assertThat(serializedBytes.length) + .as( + "Serialized bytes for sort key should be 38 bytes (34 UUID text + 4 byte integer of string length") + .isEqualTo(38); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = serializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..012654603b04 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER); + TypeSerializerSnapshot<SortKey> snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer<SortKey> restoredSerializer = snapshot.restoreSerializer(); + + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + + @Test + public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot<SortKey> original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} diff --git a/flink/v1.18/flink/src/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/TableSourceTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java new file mode 100644 index 000000000000..32c81d9465a4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/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 bc7194e38088..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 @@ -24,12 +24,12 @@ import java.util.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestFlinkSourceConfig extends TestFlinkTableSource { +public class TestFlinkSourceConfig extends TableSourceTestBase { private static final String TABLE = "test_table"; - @Test + @TestTemplate public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) @@ -37,7 +37,7 @@ public void testFlinkSessionConfig() { .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } - @Test + @TestTemplate public void testFlinkHintConfig() { List<Row> result = sql( @@ -46,10 +46,13 @@ public void testFlinkHintConfig() { assertThat(result).hasSize(3); } - @Test + @TestTemplate public void testReadOptionHierarchy() { getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List<Row> 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/TestFlinkTableSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index 01bab6d063fd..18528c789114 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -21,78 +21,16 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; -import java.io.IOException; import java.util.List; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestFlinkTableSource extends TestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - } - - @Test +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate public void testLimitPushDown() { assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) @@ -121,7 +59,7 @@ public void testLimitPushDown() { assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } - @Test + @TestTemplate public void testNoFilterPushDown() { String sql = String.format("SELECT * FROM %s ", TABLE_NAME); List<Row> result = sql(sql); @@ -133,7 +71,7 @@ public void testNoFilterPushDown() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDownEqual() { String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -147,7 +85,7 @@ public void testFilterPushDownEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); @@ -156,7 +94,7 @@ public void testFilterPushDownEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownEqualLiteralOnLeft() { String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -170,7 +108,7 @@ public void testFilterPushDownEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqual() { String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") != 1"; @@ -187,7 +125,7 @@ public void testFilterPushDownNoEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); @@ -196,7 +134,7 @@ public void testFilterPushDownNoEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownAnd() { String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); @@ -211,7 +149,7 @@ public void testFilterPushDownAnd() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownOr() { String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; @@ -229,7 +167,7 @@ public void testFilterPushDownOr() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThan() { String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 1"; @@ -247,7 +185,7 @@ public void testFilterPushDownGreaterThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); @@ -256,7 +194,7 @@ public void testFilterPushDownGreaterThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanLiteralOnLeft() { String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 3"; @@ -274,7 +212,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqual() { String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 2"; @@ -292,7 +230,7 @@ public void testFilterPushDownGreaterThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); @@ -301,7 +239,7 @@ public void testFilterPushDownGreaterThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 2"; @@ -319,7 +257,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThan() { String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 2"; @@ -334,7 +272,7 @@ public void testFilterPushDownLessThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); @@ -343,7 +281,7 @@ public void testFilterPushDownLessThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanLiteralOnLeft() { String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 2"; @@ -358,7 +296,7 @@ public void testFilterPushDownLessThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqual() { String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 1"; @@ -373,7 +311,7 @@ public void testFilterPushDownLessThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); @@ -382,7 +320,7 @@ public void testFilterPushDownLessThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualLiteralOnLeft() { String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 3"; @@ -397,7 +335,7 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; @@ -413,7 +351,7 @@ public void testFilterPushDownIn() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownInNull() { String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); @@ -430,7 +368,7 @@ public void testFilterPushDownInNull() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); @@ -444,7 +382,7 @@ public void testFilterPushDownNotIn() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List<Row> resultGT = sql(sqlNotInNull); @@ -455,7 +393,7 @@ public void testFilterPushDownNotInNull() { .isNull(); } - @Test + @TestTemplate public void testFilterPushDownIsNotNull() { String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); String expectedFilter = "not_null(ref(name=\"data\"))"; @@ -473,7 +411,7 @@ public void testFilterPushDownIsNotNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIsNull() { String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); String expectedFilter = "is_null(ref(name=\"data\"))"; @@ -488,7 +426,7 @@ public void testFilterPushDownIsNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); @@ -503,7 +441,7 @@ public void testFilterPushDownNot() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); @@ -522,7 +460,7 @@ public void testFilterPushDownBetween() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownNotBetween() { String sqlNotBetween = String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); @@ -538,7 +476,7 @@ public void testFilterPushDownNotBetween() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLike() { String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; @@ -565,7 +503,7 @@ public void testFilterPushDownLike() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; @@ -604,7 +542,7 @@ public void testFilterNotPushDownLike() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDown2Literal() { String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); List<Row> result = sql(sql2Literal); @@ -616,7 +554,7 @@ public void testFilterPushDown2Literal() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testSqlParseNaN() { // todo add some test case to test NaN } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -130,11 +128,8 @@ protected List<Row> run( sourceBuilder.properties(options); DataStream<Row> stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 0337f3597053..4e649d15b1ce 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -18,10 +18,10 @@ */ package org.apache.iceberg.flink.source; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Locale; import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -30,10 +30,12 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.AvroSchemaUtil; @@ -43,68 +45,64 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedGenericRecord { + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } - private final FileFormat fileFormat; - private final int parallelism; + @Parameter(index = 0) + private FileFormat fileFormat; - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } + @Parameter(index = 1) + private int parallelism; - @Test + @Parameter(index = 2) + private boolean useConverter; + + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { String dateStr = "2020-03-20"; Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); @@ -112,19 +110,19 @@ public void testPartitionedTable() throws Exception { expectedRecords.get(i).setField("dt", dateStr); } - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testProjection() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); // select the "data" field (fieldId == 1) Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); @@ -149,29 +147,20 @@ private List<Row> run( Configuration config = new Configuration(); config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { + try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { tableLoader.open(); table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder<GenericRecord> sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder<GenericRecord> sourceBuilder = - IcebergSource.<GenericRecord>builder() - .tableLoader(catalogResource.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -179,7 +168,6 @@ private List<Row> run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -200,4 +188,35 @@ private List<Row> run( return Lists.newArrayList(iter); } } + + private IcebergSource.Builder<GenericRecord> createSourceBuilderWithReaderFunction( + Table table, Schema projected, List<Expression> filters, Configuration config) { + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + projected, + null, + false, + table.io(), + table.encryption(), + filters); + + return IcebergSource.<GenericRecord>builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder<GenericRecord> createSourceBuilderWithConverter( + Table table, Schema readSchema, Configuration config) { + AvroGenericRecordConverter converter = + AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); + return IcebergSource.forOutputType(converter) + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 5765b73a1f63..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 @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -38,7 +39,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; @@ -46,45 +48,43 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceContinuous { public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(METRIC_REPORTER); + @TempDir protected Path temporaryFolder; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private final AtomicLong randomSeed = new AtomicLong(0L); @Test public void testTableScanThenIncremental() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -97,27 +97,27 @@ public void testTableScanThenIncremental() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -126,22 +126,24 @@ public void testTableScanThenIncremental() throws Exception { @Test public void testTableScanThenIncrementalAfterExpiration() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot2 List<Record> batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - Assert.assertEquals(1, tableResource.table().history().size()); + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); ScanContext scanContext = ScanContext.builder() @@ -150,8 +152,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { @@ -159,17 +161,17 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List<Record> initialRecords = Lists.newArrayList(); initialRecords.addAll(batch1); initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -178,16 +180,18 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { @Test public void testEarliestSnapshot() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -202,43 +206,46 @@ public void testEarliestSnapshot() throws Exception { List<Row> result1 = waitForResult(iter, 4); List<Record> combinedBatch0AndBatch1 = Lists.newArrayList(batch0); combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } } @Test - public void testLatestSnapshot() throws Exception { + public void testLatestSnapshot(@InjectClusterClient ClusterClient<?> clusterClient) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -252,29 +259,29 @@ public void testLatestSnapshot() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // we want to make sure job is running first so that enumerator can // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + waitUntilJobIsRunning(clusterClient); // inclusive behavior for starting snapshot List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -283,19 +290,21 @@ public void testLatestSnapshot() throws Exception { @Test public void testSpecificSnapshotId() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); ScanContext scanContext = ScanContext.builder() @@ -308,25 +317,25 @@ public void testSpecificSnapshotId() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -335,22 +344,24 @@ public void testSpecificSnapshotId() throws Exception { @Test public void testSpecificSnapshotTimestamp() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); // sleep for 2 ms to make sure snapshot1 has a higher timestamp value Thread.sleep(2); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); ScanContext scanContext = ScanContext.builder() @@ -364,25 +375,25 @@ public void testSpecificSnapshotTimestamp() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // consume data from snapshot1 List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -392,27 +403,30 @@ public void testSpecificSnapshotTimestamp() throws Exception { public void testReadingFromBranch() throws Exception { String branch = "b1"; GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); List<Record> batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchBase); // create branch - tableResource + TABLE_EXTENSION .table() .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) .commit(); // snapshot1 to branch List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch1); // snapshot2 to branch List<Record> batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch2); List<Record> branchExpectedRecords = Lists.newArrayList(); @@ -432,25 +446,26 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); // snapshot3 to branch List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); // snapshot4 to branch List<Record> batch4 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch4); List<Row> result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); } // read only from main branch. Should contain only the first snapshot @@ -463,14 +478,14 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); List<Record> batchMain2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchMain2); resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); } } @@ -479,7 +494,7 @@ public void testValidation() { assertThatThrownBy( () -> IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(true) .endTag("tag") @@ -495,7 +510,7 @@ private DataStream<Row> createStream(ScanContext scanContext) throws Exception { DataStream<Row> stream = env.fromSource( IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(scanContext.isStreaming()) .streamingStartingStrategy(scanContext.streamingStartingStrategy()) @@ -507,7 +522,7 @@ private DataStream<Row> createStream(ScanContext scanContext) throws Exception { WatermarkStrategy.noWatermarks(), "icebergSource", TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); return stream; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index a378a0b93be3..938ae4d9bb0a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Path; import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,6 +31,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; @@ -41,7 +43,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -51,53 +56,67 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; - +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 120) public class TestIcebergSourceFailover { // Parallelism higher than 1, but lower than the number of splits used by some of our tests // The goal is to allow some splits to remain in the enumerator when restoring the state private static final int PARALLELISM = 2; private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } protected IcebergSource.Builder<RowData> sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) // Prevent combining splits .set( @@ -119,13 +138,13 @@ protected void assertRecords(Table table, List<Record> expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Disabled("Disabled for now as it is flaky on CI") @Test - public void testBoundedWithSavepoint() throws Exception { + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient<?> clusterClient) + throws Exception { List<Record> expectedRecords = Lists.newArrayList(); - Table sinkTable = sinkTableResource.table(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List<Record> records = generateRecords(2, i); expectedRecords.addAll(records); @@ -141,13 +160,8 @@ public void testBoundedWithSavepoint() throws Exception { // Write something, but do not finish before checkpoint is created RecordCounterToWait.waitForCondition(); CompletableFuture<String> savepoint = - miniClusterResource - .getClusterClient() - .stopWithSavepoint( - jobId, - false, - TEMPORARY_FOLDER.newFolder().toPath().toString(), - SavepointFormatType.CANONICAL); + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint @@ -170,19 +184,21 @@ public void testBoundedWithSavepoint() throws Exception { @Test public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); } - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { List<Record> expectedRecords = Lists.newArrayList(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List<Record> records = generateRecords(2, i); expectedRecords.addAll(records); @@ -197,29 +213,27 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio JobID jobId = jobClient.getJobID(); RecordCounterToWait.waitForCondition(); - triggerFailover( - failoverType, - jobId, - RecordCounterToWait::continueProcessing, - miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } @Test public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); } - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); List<Record> expectedRecords = Lists.newArrayList(); List<Record> batch = generateRecords(2, 0); @@ -247,8 +261,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); @@ -260,13 +274,13 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep expectedRecords.addAll(records); dataAppender.appendToTable(records); if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, () -> {}, miniCluster); } } // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { @@ -286,8 +300,8 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); } @@ -295,6 +309,20 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ + private static void runTestWithNewMiniCluster(ThrowingConsumer<MiniCluster, Exception> testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + private enum FailoverType { NONE, TM, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index cde39ec98d61..4f61d2f7308a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -49,11 +50,24 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + @Override protected IcebergSource.Builder<RowData> sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) // Prevent combining splits diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream<Row> dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector<Row> collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator<Row> iterator = collector.getOutput()) { + List<Row> result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 75f0a785a8c5..548940a842ce 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -53,7 +53,12 @@ public class TestIcebergSourceSql extends TestSqlBase { public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 8341c8074f21..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,9 +18,10 @@ */ 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; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -37,6 +38,7 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.InMemoryReporter; @@ -49,24 +51,24 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; @@ -74,25 +76,24 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int RECORD_NUM_FOR_2_SPLITS = 200; private static final ConcurrentMap<Long, Integer> WINDOWS = Maps.newConcurrentMap(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) .withHaLeadershipControl() .build()); - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); /** * This is an integration test for watermark handling and windowing. Integration testing the @@ -259,7 +260,7 @@ public void apply( * the readers continue reading. */ @Test - public void testThrottling() throws Exception { + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { GenericAppenderHelper dataAppender = appender(); // Generate records in advance @@ -310,8 +311,7 @@ public void testThrottling() throws Exception { try (CloseableIterator<RowData> resultIterator = stream.collectAsync()) { JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); // Insert the first data into the table dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); @@ -353,7 +353,7 @@ public void testThrottling() throws Exception { protected IcebergSource<RowData> source() { return IcebergSource.<RowData>builder() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) .splitSize(100L) @@ -377,7 +377,7 @@ protected Record generateRecord(int minutes, String str) { private Optional<Gauge<Long>> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() + return REPORTER.findMetrics(jobID, metricsName).values().stream() .map(m -> (Gauge<Long>) m) .filter(m -> m.getValue() == withValue) .findFirst(); @@ -389,7 +389,7 @@ private GenericAppenderHelper appender() { hadoopConf.set("write.parquet.page-size-bytes", "64"); hadoopConf.set("write.parquet.row-group-size-bytes", "64"); return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); } private static RowData row(long time, long count) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..564e8139e6cc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.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.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static 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<Row> 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<Row> 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<Row, Row> { + @Override + public Row map(Row row) throws Exception { + // Simulate slow subtask 0 with attempt 0 + if (getRuntimeContext().getIndexOfThisSubtask() == 0 + && 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); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + + // 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/TestProjectMetaColumn.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java index 25ecec23d216..ce9054ad49b6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowDelta; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; @@ -40,20 +47,19 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestProjectMetaColumn { - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Iterable<Object[]> parameters() { return Lists.newArrayList( new Object[] {FileFormat.PARQUET}, @@ -61,13 +67,9 @@ public static Iterable<Object[]> parameters() { new Object[] {FileFormat.AVRO}); } - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, @@ -89,7 +91,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); + assertThat(rowData).isInstanceOf(GenericRowData.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); @@ -97,20 +99,20 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); } - @Test + @TestTemplate public void testV1SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(1); } - @Test + @TestTemplate public void testV2SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(2); } - @Test + @TestTemplate public void testV2RemoveMetaColumn() throws Exception { // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); @@ -132,7 +134,7 @@ public void testV2RemoveMetaColumn() throws Exception { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); + assertThat(rowData).isInstanceOf(RowDataProjection.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java index 317301260f66..b701419a7499 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSourceUtil { @Test @@ -30,31 +31,31 @@ public void testInferedParallelism() throws IOException { Configuration configuration = new Configuration(); // Empty table, infer parallelism should be at least 1 int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits // num : 2 parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + assertThat(parallelism).isEqualTo(2); // 2 splits and limit is 1 , max infer parallelism is default 100, // which is greater than splits num and limit, the parallelism is the limit value : 1 parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : // 1 parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); } } 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<List<Record>> 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<RowData> dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List<RowData> 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/build.gradle b/flink/v1.19/build.gradle index a0954942f0e9..6327ebf99dbd 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -119,7 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core - testImplementation libs.junit.vintage.engine + testImplementation libs.sqlite.jdbc } test { 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..24cad2669dbe 100644 --- a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -27,6 +29,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -66,6 +70,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator<StructLike> SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -82,10 +88,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List<Integer> keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; @@ -133,7 +140,7 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + return prefix + new String(buffer, StandardCharsets.US_ASCII); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public <E extends Enum<E>> EnumConfParser<E> enumConfParser(Class<E> enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser<DoubleConfParser, Double> { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser<StringConfParser, String> { private String defaultValue; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption<String> DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption<String> RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption<Double> RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption<String> BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); 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/maintenance/operator/JdbcLockFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java new file mode 100644 index 000000000000..f22be33aeae6 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java @@ -0,0 +1,319 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JdbcLockFactory implements TriggerLockFactory { + private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); + + @VisibleForTesting + static final String INIT_LOCK_TABLES_PROPERTY = "flink-maintenance.lock.jdbc.init-lock-tables"; + + private static final String LOCK_TABLE_NAME = "flink_maintenance_lock"; + private static final int LOCK_ID_MAX_LENGTH = 100; + private static final String CREATE_LOCK_TABLE_SQL = + String.format( + "CREATE TABLE %s " + + "(LOCK_TYPE CHAR(1) NOT NULL, " + + "LOCK_ID VARCHAR(%s) NOT NULL, " + + "INSTANCE_ID CHAR(36) NOT NULL, PRIMARY KEY (LOCK_TYPE, LOCK_ID))", + LOCK_TABLE_NAME, LOCK_ID_MAX_LENGTH); + + private static final String CREATE_LOCK_SQL = + String.format( + "INSERT INTO %s (LOCK_TYPE, LOCK_ID, INSTANCE_ID) VALUES (?, ?, ?)", LOCK_TABLE_NAME); + private static final String GET_LOCK_SQL = + String.format("SELECT INSTANCE_ID FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=?", LOCK_TABLE_NAME); + private static final String DELETE_LOCK_SQL = + String.format( + "DELETE FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=? AND INSTANCE_ID=?", LOCK_TABLE_NAME); + + private final String uri; + private final String lockId; + private final Map<String, String> properties; + private transient JdbcClientPool pool; + + /** + * Creates a new {@link TriggerLockFactory}. The lockId should be unique between the users of the + * same uri. + * + * @param uri of the jdbc connection + * @param lockId which should indentify the job and the table + * @param properties used for creating the jdbc connection pool + */ + public JdbcLockFactory(String uri, String lockId, Map<String, String> properties) { + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + Preconditions.checkNotNull(properties, "Properties map is required"); + Preconditions.checkArgument( + lockId.length() < LOCK_ID_MAX_LENGTH, + "Invalid prefix length: lockId should be shorter than %s", + LOCK_ID_MAX_LENGTH); + this.uri = uri; + this.lockId = lockId; + this.properties = properties; + } + + @Override + public void open() { + this.pool = new JdbcClientPool(1, uri, properties); + + if (PropertyUtil.propertyAsBoolean(properties, INIT_LOCK_TABLES_PROPERTY, false)) { + initializeLockTables(); + } + } + + /** Only used in testing to share the jdbc pool */ + @VisibleForTesting + void open(JdbcLockFactory other) { + this.pool = other.pool; + } + + @Override + public Lock createLock() { + return new JdbcLock(pool, lockId, Type.MAINTENANCE); + } + + @Override + public Lock createRecoveryLock() { + return new JdbcLock(pool, lockId, Type.RECOVERY); + } + + @Override + public void close() throws IOException { + pool.close(); + } + + private void initializeLockTables() { + LOG.debug("Creating database tables (if missing) to store table maintenance locks"); + try { + pool.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + LOCK_TABLE_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + LOG.debug("Flink maintenance lock table already exists"); + return true; + } + + LOG.info("Creating Flink maintenance lock table {}", LOCK_TABLE_NAME); + return conn.prepareStatement(CREATE_LOCK_TABLE_SQL).execute(); + }); + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC table maintenance lock"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + private static class JdbcLock implements TriggerLockFactory.Lock { + private final JdbcClientPool pool; + private final String lockId; + private final Type type; + + private JdbcLock(JdbcClientPool pool, String lockId, Type type) { + this.pool = pool; + this.lockId = lockId; + this.type = type; + } + + @Override + public boolean tryLock() { + if (isHeld()) { + LOG.info("Lock is already held for {}", this); + return false; + } + + String newInstanceId = UUID.randomUUID().toString(); + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(CREATE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, newInstanceId); + int count = sql.executeUpdate(); + LOG.info( + "Created {} lock with instanceId {} with row count {}", + this, + newInstanceId, + count); + return count == 1; + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during tryLock"); + } catch (SQLException e) { + // SQL exception happened when creating the lock. Check if the lock creation was + // successful behind the scenes. + if (newInstanceId.equals(instanceId())) { + return true; + } else { + throw new UncheckedSQLException(e, "Failed to create %s lock", this); + } + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public boolean isHeld() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + return rs.next(); + } + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to check the state of the lock %s", this); + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public void unlock() { + try { + // Possible concurrency issue: + // - `unlock` and `tryLock` happens at the same time when there is an existing lock + // + // Steps: + // 1. `unlock` removes the lock in the database, but there is a temporary connection failure + // 2. `lock` founds that there is no lock, so creates a new lock + // 3. `unlock` retires the lock removal and removes the new lock + // + // To prevent the situation above we fetch the current lockId, and remove the lock + // only with the given id. + String instanceId = instanceId(); + + if (instanceId != null) { + pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(DELETE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, instanceId); + long count = sql.executeUpdate(); + LOG.info( + "Deleted {} lock with instanceId {} with row count {}", + this, + instanceId, + count); + } catch (SQLException e) { + // SQL exception happened when deleting lock information + throw new UncheckedSQLException( + e, "Failed to delete %s lock with instanceId %s", this, instanceId); + } + + return null; + }); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + // SQL exception happened when getting/updating lock information + throw new UncheckedSQLException(e, "Failed to remove lock %s", this); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("type", type).add("lockId", lockId).toString(); + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + private String instanceId() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + if (rs.next()) { + return rs.getString(1); + } else { + return null; + } + } + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + } + } + + private enum Type { + MAINTENANCE("m"), + RECOVERY("r"); + + private final String key; + + Type(String key) { + this.key = key; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java index d74b2349b1de..89efffa15f16 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java @@ -43,7 +43,7 @@ /** Monitors an Iceberg table for changes */ @Internal -public class MonitorSource extends SingleThreadedIteratorSource<TableChange> { +class MonitorSource extends SingleThreadedIteratorSource<TableChange> { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ public class MonitorSource extends SingleThreadedIteratorSource<TableChange> { * @param rateLimiterStrategy limits the frequency the table is checked * @param maxReadBack sets the number of snapshots read before stopping change collection */ - public MonitorSource( + MonitorSource( TableLoader tableLoader, RateLimiterStrategy rateLimiterStrategy, long maxReadBack) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(rateLimiterStrategy, "Rate limiter strategy should no be null"); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java index 452ed80ed0e5..773b34b6c495 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -29,19 +29,29 @@ /** Event describing changes in an Iceberg table */ @Internal class TableChange { - private int dataFileNum; - private int deleteFileNum; - private long dataFileSize; - private long deleteFileSize; - private int commitNum; + private int dataFileCount; + private long dataFileSizeInBytes; + private int posDeleteFileCount; + private long posDeleteRecordCount; + private int eqDeleteFileCount; + private long eqDeleteRecordCount; + private int commitCount; 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; + int dataFileCount, + long dataFileSizeInBytes, + int posDeleteFileCount, + long posDeleteRecordCount, + int eqDeleteFileCount, + long eqDeleteRecordCount, + int commitCount) { + this.dataFileCount = dataFileCount; + this.dataFileSizeInBytes = dataFileSizeInBytes; + this.posDeleteFileCount = posDeleteFileCount; + this.posDeleteRecordCount = posDeleteRecordCount; + this.eqDeleteFileCount = eqDeleteFileCount; + this.eqDeleteRecordCount = eqDeleteRecordCount; + this.commitCount = commitCount; } TableChange(Snapshot snapshot, FileIO io) { @@ -50,63 +60,96 @@ class TableChange { dataFiles.forEach( dataFile -> { - this.dataFileNum++; - this.dataFileSize += dataFile.fileSizeInBytes(); + this.dataFileCount++; + this.dataFileSizeInBytes += dataFile.fileSizeInBytes(); }); deleteFiles.forEach( deleteFile -> { - this.deleteFileNum++; - this.deleteFileSize += deleteFile.fileSizeInBytes(); + switch (deleteFile.content()) { + case POSITION_DELETES: + this.posDeleteFileCount++; + this.posDeleteRecordCount += deleteFile.recordCount(); + break; + case EQUALITY_DELETES: + this.eqDeleteFileCount++; + this.eqDeleteRecordCount += deleteFile.recordCount(); + break; + default: + throw new IllegalArgumentException("Unexpected delete file content: " + deleteFile); + } }); - this.commitNum = 1; + this.commitCount = 1; } static TableChange empty() { - return new TableChange(0, 0, 0L, 0L, 0); + return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } - int dataFileNum() { - return dataFileNum; + static Builder builder() { + return new Builder(); } - int deleteFileNum() { - return deleteFileNum; + int dataFileCount() { + return dataFileCount; } - long dataFileSize() { - return dataFileSize; + long dataFileSizeInBytes() { + return dataFileSizeInBytes; } - long deleteFileSize() { - return deleteFileSize; + int posDeleteFileCount() { + return posDeleteFileCount; } - public int commitNum() { - return commitNum; + long posDeleteRecordCount() { + return posDeleteRecordCount; + } + + int eqDeleteFileCount() { + return eqDeleteFileCount; + } + + long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public int commitCount() { + return commitCount; } public void merge(TableChange other) { - this.dataFileNum += other.dataFileNum; - this.deleteFileNum += other.deleteFileNum; - this.dataFileSize += other.dataFileSize; - this.deleteFileSize += other.deleteFileSize; - this.commitNum += other.commitNum; + this.dataFileCount += other.dataFileCount; + this.dataFileSizeInBytes += other.dataFileSizeInBytes; + this.posDeleteFileCount += other.posDeleteFileCount; + this.posDeleteRecordCount += other.posDeleteRecordCount; + this.eqDeleteFileCount += other.eqDeleteFileCount; + this.eqDeleteRecordCount += other.eqDeleteRecordCount; + this.commitCount += other.commitCount; } TableChange copy() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("dataFileNum", dataFileNum) - .add("deleteFileNum", deleteFileNum) - .add("dataFileSize", dataFileSize) - .add("deleteFileSize", deleteFileSize) - .add("commitNum", commitNum) + .add("dataFileCount", dataFileCount) + .add("dataFileSizeInBytes", dataFileSizeInBytes) + .add("posDeleteFileCount", posDeleteFileCount) + .add("posDeleteRecordCount", posDeleteRecordCount) + .add("eqDeleteFileCount", eqDeleteFileCount) + .add("eqDeleteRecordCount", eqDeleteRecordCount) + .add("commitCount", commitCount) .toString(); } @@ -119,15 +162,82 @@ public boolean equals(Object other) { } TableChange that = (TableChange) other; - return dataFileNum == that.dataFileNum - && deleteFileNum == that.deleteFileNum - && dataFileSize == that.dataFileSize - && deleteFileSize == that.deleteFileSize - && commitNum == that.commitNum; + return dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes + && posDeleteFileCount == that.posDeleteFileCount + && posDeleteRecordCount == that.posDeleteRecordCount + && eqDeleteFileCount == that.eqDeleteFileCount + && eqDeleteRecordCount == that.eqDeleteRecordCount + && commitCount == that.commitCount; } @Override public int hashCode() { - return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return Objects.hash( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } + + static class Builder { + private int dataFileCount = 0; + private long dataFileSizeInBytes = 0L; + private int posDeleteFileCount = 0; + private long posDeleteRecordCount = 0L; + private int eqDeleteFileCount = 0; + private long eqDeleteRecordCount = 0L; + private int commitCount = 0; + + private Builder() {} + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + public TableChange build() { + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java new file mode 100644 index 000000000000..ec0fd920c34b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +public class TableMaintenanceMetrics { + public static final String GROUP_KEY = "maintenanceTask"; + public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + + // TriggerManager metrics + public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; + public static final String CONCURRENT_RUN_THROTTLED = "concurrentRunThrottled"; + public static final String TRIGGERED = "triggered"; + public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + + private TableMaintenanceMetrics() { + // do not instantiate + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java new file mode 100644 index 000000000000..85c6c8dbdd55 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +@Internal +class Trigger { + private final long timestamp; + private final SerializableTable table; + private final Integer taskId; + private final boolean isRecovery; + + private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + this.timestamp = timestamp; + this.table = table; + this.taskId = taskId; + this.isRecovery = isRecovery; + } + + static Trigger create(long timestamp, SerializableTable table, int taskId) { + return new Trigger(timestamp, table, taskId, false); + } + + static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, null, true); + } + + long timestamp() { + return timestamp; + } + + SerializableTable table() { + return table; + } + + Integer taskId() { + return taskId; + } + + boolean isRecovery() { + return isRecovery; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("timestamp", timestamp) + .add("table", table == null ? null : table.name()) + .add("taskId", taskId) + .add("isRecovery", isRecovery) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java new file mode 100644 index 000000000000..dba33b22a42a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Serializable; +import java.time.Duration; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TriggerEvaluator implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); + private final List<Predicate> predicates; + + private TriggerEvaluator(List<Predicate> predicates) { + Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 condition."); + + this.predicates = predicates; + } + + boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { + boolean result = + predicates.stream().anyMatch(p -> p.evaluate(event, lastTimeMs, currentTimeMs)); + LOG.debug( + "Checking event: {}, at {}, last: {} with result: {}", + event, + currentTimeMs, + lastTimeMs, + result); + return result; + } + + static class Builder implements Serializable { + private Integer dataFileCount; + private Long dataFileSizeInBytes; + private Integer posDeleteFileCount; + private Long posDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer commitCount; + private Duration timeout; + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long neDataFileSizeInBytes) { + this.dataFileSizeInBytes = neDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + Builder timeout(Duration newTimeout) { + this.timeout = newTimeout; + return this; + } + + TriggerEvaluator build() { + List<Predicate> predicates = Lists.newArrayList(); + if (dataFileCount != null) { + predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); + } + + if (dataFileSizeInBytes != null) { + predicates.add( + (change, unused, unused2) -> change.dataFileSizeInBytes() >= dataFileSizeInBytes); + } + + if (posDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteFileCount() >= posDeleteFileCount); + } + + if (posDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteRecordCount() >= posDeleteRecordCount); + } + + if (eqDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteFileCount() >= eqDeleteFileCount); + } + + if (eqDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteRecordCount() >= eqDeleteRecordCount); + } + + if (commitCount != null) { + predicates.add((change, unused, unused2) -> change.commitCount() >= commitCount); + } + + if (timeout != null) { + predicates.add( + (change, lastTimeMs, currentTimeMs) -> + currentTimeMs - lastTimeMs >= timeout.toMillis()); + } + + return new TriggerEvaluator(predicates); + } + } + + private interface Predicate extends Serializable { + boolean evaluate(TableChange event, long lastTimeMs, long currentTimeMs); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java new file mode 100644 index 000000000000..329223d27ccf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Closeable; +import java.io.Serializable; +import org.apache.flink.annotation.Experimental; + +/** Lock interface for handling locks for the Flink Table Maintenance jobs. */ +@Experimental +public interface TriggerLockFactory extends Serializable, Closeable { + void open(); + + Lock createLock(); + + Lock createRecoveryLock(); + + interface Lock { + /** + * Tries to acquire a lock with a given key. Anyone already holding a lock would prevent + * acquiring this lock. Not reentrant. + * + * <p>Called by {@link TriggerManager}. Implementations could assume that are no concurrent + * calls for this method. + * + * @return <code>true</code> if the lock is acquired by this job, <code>false</code> if the lock + * is already held by someone + */ + boolean tryLock(); + + /** + * Checks if the lock is already taken. + * + * @return <code>true</code> if the lock is held by someone + */ + boolean isHeld(); + + // TODO: Fix the link to the LockRemover when we have a final name and implementation + /** + * Releases the lock. Should not fail if the lock is not held by anyone. + * + * <p>Called by LockRemover. Implementations could assume that are no concurrent calls for this + * method. + */ + void unlock(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java new file mode 100644 index 000000000000..f4c3c1d47cf7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} messages which are + * calculated based on the incoming {@link TableChange} messages. The TriggerManager keeps track of + * the changes since the last run of the Maintenance Tasks and triggers a new run based on the + * result of the {@link TriggerEvaluator}. + * + * <p>The TriggerManager prevents overlapping Maintenance Task runs using {@link + * TriggerLockFactory.Lock}. The current implementation only handles conflicts within a single job. + * Users should avoid scheduling maintenance for the same table in different Flink jobs. + * + * <p>The TriggerManager should run as a global operator. {@link KeyedProcessFunction} is used, so + * the timer functions are available, but the key is not used. + */ +@Internal +class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, Trigger> + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); + + private final TableLoader tableLoader; + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + private final List<TriggerEvaluator> evaluators; + private final long minFireDelayMs; + private final long lockCheckDelayMs; + private transient Counter rateLimiterTriggeredCounter; + private transient Counter concurrentRunThrottledCounter; + private transient Counter nothingToTriggerCounter; + private transient List<Counter> triggerCounters; + private transient ValueState<Long> nextEvaluationTimeState; + private transient ListState<TableChange> accumulatedChangesState; + private transient ListState<Long> lastTriggerTimesState; + private transient Long nextEvaluationTime; + private transient List<TableChange> accumulatedChanges; + private transient List<Long> lastTriggerTimes; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient boolean shouldRestoreTasks = false; + private transient boolean inited = false; + // To keep the task scheduling fair we keep the last triggered task position in memory. + // If we find a task to trigger, then we run it, but after it is finished, we start from the given + // position to prevent "starvation" of the tasks. + // When there is nothing to trigger, we start from the beginning, as the order of the tasks might + // be important (RewriteDataFiles first, and then RewriteManifestFiles later) + private transient int startsFrom = 0; + private transient boolean triggered = false; + + TriggerManager( + TableLoader tableLoader, + TriggerLockFactory lockFactory, + List<String> taskNames, + List<TriggerEvaluator> evaluators, + long minFireDelayMs, + long lockCheckDelayMs) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + Preconditions.checkArgument( + evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); + Preconditions.checkArgument( + taskNames.size() == evaluators.size(), "Provide a name and evaluator for all of the tasks"); + Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); + Preconditions.checkArgument( + lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); + + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskNames = taskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.rateLimiterTriggeredCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); + this.concurrentRunThrottledCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); + this.nothingToTriggerCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); + this.triggerCounters = + taskNames.stream() + .map( + name -> + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.TRIGGERED)) + .collect(Collectors.toList()); + + this.nextEvaluationTimeState = + getRuntimeContext() + .getState(new ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG)); + this.accumulatedChangesState = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>( + "triggerManagerAccumulatedChange", TypeInformation.of(TableChange.class))); + this.lastTriggerTimesState = + getRuntimeContext() + .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); + + tableLoader.open(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (inited) { + // Only store state if initialized + nextEvaluationTimeState.update(nextEvaluationTime); + accumulatedChangesState.update(accumulatedChanges); + lastTriggerTimesState.update(lastTriggerTimes); + LOG.info( + "Storing state: nextEvaluationTime {}, accumulatedChanges {}, lastTriggerTimes {}", + nextEvaluationTime, + accumulatedChanges, + lastTriggerTimes); + } else { + LOG.info("Not initialized, state is not stored"); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOG.info("Initializing state restored: {}", context.isRestored()); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + if (context.isRestored()) { + shouldRestoreTasks = true; + } + } + + @Override + public void processElement(TableChange change, Context ctx, Collector<Trigger> out) + throws Exception { + init(out, ctx.timerService()); + + accumulatedChanges.forEach(tableChange -> tableChange.merge(change)); + + long current = ctx.timerService().currentProcessingTime(); + if (nextEvaluationTime == null) { + checkAndFire(current, ctx.timerService(), out); + } else { + LOG.info( + "Trigger manager rate limiter triggered current: {}, next: {}, accumulated changes: {}", + current, + nextEvaluationTime, + accumulatedChanges); + rateLimiterTriggeredCounter.inc(); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> out) throws Exception { + init(out, ctx.timerService()); + this.nextEvaluationTime = null; + checkAndFire(ctx.timerService().currentProcessingTime(), ctx.timerService(), out); + } + + @Override + public void close() throws IOException { + tableLoader.close(); + lockFactory.close(); + } + + private void checkAndFire(long current, TimerService timerService, Collector<Trigger> out) { + if (shouldRestoreTasks) { + if (recoveryLock.isHeld()) { + // Recovered tasks in progress. Skip trigger check + LOG.debug("The recovery lock is still held at {}", current); + schedule(timerService, current + lockCheckDelayMs); + return; + } else { + LOG.info("The recovery is finished at {}", current); + shouldRestoreTasks = false; + } + } + + Integer taskToStart = + nextTrigger(evaluators, accumulatedChanges, lastTriggerTimes, current, startsFrom); + if (taskToStart == null) { + // Nothing to execute + if (!triggered) { + nothingToTriggerCounter.inc(); + LOG.debug("Nothing to execute at {} for collected: {}", current, accumulatedChanges); + } else { + LOG.debug("Execution check finished"); + } + + // Next time start from the beginning + startsFrom = 0; + triggered = false; + return; + } + + if (lock.tryLock()) { + TableChange change = accumulatedChanges.get(taskToStart); + SerializableTable table = + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()); + out.collect(Trigger.create(current, table, taskToStart)); + LOG.debug("Fired event with time: {}, collected: {} for {}", current, change, table.name()); + triggerCounters.get(taskToStart).inc(); + accumulatedChanges.set(taskToStart, TableChange.empty()); + lastTriggerTimes.set(taskToStart, current); + schedule(timerService, current + minFireDelayMs); + startsFrom = (taskToStart + 1) % evaluators.size(); + triggered = true; + } else { + // A task is already running, waiting for it to finish + LOG.info("Failed to acquire lock. Delaying task to {}", current + lockCheckDelayMs); + + startsFrom = taskToStart; + concurrentRunThrottledCounter.inc(); + schedule(timerService, current + lockCheckDelayMs); + } + + timerService.registerProcessingTimeTimer(nextEvaluationTime); + } + + private void schedule(TimerService timerService, long time) { + this.nextEvaluationTime = time; + timerService.registerProcessingTimeTimer(time); + } + + private static Integer nextTrigger( + List<TriggerEvaluator> evaluators, + List<TableChange> changes, + List<Long> lastTriggerTimes, + long currentTime, + int startPos) { + int current = startPos; + do { + if (evaluators + .get(current) + .check(changes.get(current), lastTriggerTimes.get(current), currentTime)) { + return current; + } + + current = (current + 1) % evaluators.size(); + } while (current != startPos); + + return null; + } + + private void init(Collector<Trigger> out, TimerService timerService) throws Exception { + if (!inited) { + long current = timerService.currentProcessingTime(); + + // Initialize from state + this.nextEvaluationTime = nextEvaluationTimeState.value(); + this.accumulatedChanges = Lists.newArrayList(accumulatedChangesState.get()); + this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get()); + + // Initialize if the state was empty + if (accumulatedChanges.isEmpty()) { + for (int i = 0; i < evaluators.size(); ++i) { + accumulatedChanges.add(TableChange.empty()); + lastTriggerTimes.add(current); + } + } + + if (shouldRestoreTasks) { + // When the job state is restored, there could be ongoing tasks. + // To prevent collision with the new triggers the following is done: + // - add a recovery lock + // - fire a recovery trigger + // This ensures that the tasks of the previous trigger are executed, and the lock is removed + // in the end. The result of the 'tryLock' is ignored as an already existing lock prevents + // collisions as well. + recoveryLock.tryLock(); + out.collect(Trigger.recovery(current)); + if (nextEvaluationTime == null) { + schedule(timerService, current + minFireDelayMs); + } + } + + inited = true; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index c7e8a2dea7cb..9571efdc5268 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List<ManifestFile> manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 769af7d77140..c53431490984 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -53,15 +53,20 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; +import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -233,15 +238,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + * <p>Default is {@link StatisticsType#Auto} where initially Map statistics is used. But if + * cardinality is higher than the threshold (currently 10K) as defined in {@code + * SketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD}, statistics collection automatically switches to + * the sketch reservoir sampling. + * + * <p>Explicit set the statistics type if the default behavior doesn't work. + * + * @param type to specify the statistics type for range distribution. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rangeDistributionStatisticsType(StatisticsType type) { + if (type != null) { + writeOptions.put(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key(), type.name()); + } + return this; + } + + /** + * If sort order contains partition columns, each sort key would map to one partition and data + * file. This relative weight can avoid placing too many small files for sort keys with low + * traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means + * each key has a base weight of `2%` of the targeted traffic weight per writer task. + * + * <p>E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream + * contains events from now up to 180 days ago. With event time, traffic weight distribution + * across different days typically has a long tail pattern. Current day contains the most + * traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism + * is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer + * task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, + * the range partitioner would put all the oldest 150 days in one writer task. That writer task + * would write to 150 small files (one per day). Keeping 150 open files can potentially consume + * large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time + * can also be potentially slow. If this config is set to `0.02`. It means every sort key has a + * base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially + * avoid placing more than `50` data files (one per day) on one writer task no matter how small + * they are. + * + * <p>This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For + * {@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as + * partition columns. Otherwise, too many partitions and small files may be generated during + * write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. + * + * <p>Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -347,20 +405,23 @@ private <T> DataStreamSink<T> chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List<Integer> equalityFieldIds = checkAndGetEqualityFieldIds(); + List<Integer> equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream<RowData> distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator<WriteResult> writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + SingleOutputStreamOperator<FlinkWriteResult> writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -426,7 +487,7 @@ private <T> DataStreamSink<T> appendDummySink( } private SingleOutputStreamOperator<Void> appendCommitter( - SingleOutputStreamOperator<WriteResult> writerStream) { + SingleOutputStreamOperator<FlinkWriteResult> writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -446,8 +507,11 @@ private SingleOutputStreamOperator<Void> appendCommitter( return committerStream; } - private SingleOutputStreamOperator<WriteResult> appendWriter( - DataStream<RowData> input, RowType flinkRowType, List<Integer> equalityFieldIds) { + private SingleOutputStreamOperator<FlinkWriteResult> appendWriter( + DataStream<RowData> input, + RowType flinkRowType, + List<Integer> equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +545,13 @@ private SingleOutputStreamOperator<WriteResult> appendWriter( IcebergStreamWriter<RowData> streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator<WriteResult> writerStream = + SingleOutputStreamOperator<FlinkWriteResult> writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +561,15 @@ private SingleOutputStreamOperator<WriteResult> appendWriter( private DataStream<RowData> distributeDataStream( DataStream<RowData> input, List<Integer> equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +611,52 @@ private DataStream<RowData> distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator<StatisticsOrRecord> shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +671,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + * <p>{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + * <p>In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer<IcebergCommittable> { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..2245b36f15a7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +class IcebergCommitter implements Committer<IcebergCommittable> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + public static final WriteResult EMPTY_WRITE_RESULT = + WriteResult.builder() + .addDataFiles(Lists.newArrayList()) + .addDeleteFiles(Lists.newArrayList()) + .build(); + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private IcebergFilesCommitterMetrics committerMetrics; + private Table table; + private final TableLoader tableLoader; + private int maxContinuousEmptyCommits; + private ExecutorService workerPool; + private int continuousEmptyCheckpoints = 0; + + IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + this.tableLoader = tableLoader; + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection<CommitRequest<IcebergCommittable>> commitRequests) + throws IOException, InterruptedException { + if (commitRequests.isEmpty()) { + return; + } + + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap = Maps.newTreeMap(); + for (CommitRequest<IcebergCommittable> request : commitRequests) { + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + long maxCommittedCheckpointId = + SinkUtil.getMaxCommittedCheckpointId(table, last.jobId(), last.operatorId(), branch); + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + NavigableMap<Long, CommitRequest<IcebergCommittable>> uncommitted = + commitRequestMap.tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests(uncommitted, last.jobId(), last.operatorId()); + } + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link + * org.apache.iceberg.flink.sink.DeltaManifests} ordered by the checkpointId, and writing the new + * snapshot to the Iceberg table. The {@link org.apache.iceberg.SnapshotSummary} will contain the + * jobId, snapshotId, checkpointId so in case of job restart we can identify which changes are + * committed, and which are still waiting for the commit. + * + * @param commitRequestMap The checkpointId to {@link CommitRequest} map of the changes to commit + * @param newFlinkJobId The jobId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @param operatorId The operatorId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @throws IOException On commit failure + */ + private void commitPendingRequests( + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap, + String newFlinkJobId, + String operatorId) + throws IOException { + long checkpointId = commitRequestMap.lastKey(); + List<ManifestFile> manifests = Lists.newArrayList(); + NavigableMap<Long, WriteResult> pendingResults = Maps.newTreeMap(); + for (Map.Entry<Long, CommitRequest<IcebergCommittable>> e : commitRequestMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue().getCommittable().manifest())) { + pendingResults.put(e.getKey(), EMPTY_WRITE_RESULT); + } else { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue().getCommittable().manifest()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(summary); + } + + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void logCommitSummary(CommitSummary summary, String description) { + LOG.info( + "Preparing for commit: {} on table: {} branch: {} with summary: {}.", + description, + table, + branch, + summary); + } + + private void commitPendingResult( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId); + } + continuousEmptyCheckpoints = 0; + } else { + long checkpointId = pendingResults.lastKey(); + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void replacePartitions( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + String description = "dynamic partition overwrite"; + + logCommitSummary(summary, description); + commitOperation(dynamicOverwrite, description, newFlinkJobId, operatorId, checkpointId); + } + + private void commitDeltaTxn( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + String description = "append"; + logCommitSummary(summary, description); + // fail all commits as really its only one + commitOperation(appendFiles, description, newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry<Long, WriteResult> e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + + String description = "rowDelta"; + logCommitSummary(summary, description); + commitOperation(rowDelta, description, newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate<?> operation, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(SinkUtil.MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(SinkUtil.FLINK_JOB_ID, newFlinkJobId); + operation.set(SinkUtil.OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + if (committerMetrics != null) { + committerMetrics.commitDuration(durationMs); + } + } + + @Override + public void close() throws IOException { + tableLoader.close(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..609deb621f43 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -44,13 +44,11 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -63,7 +61,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator<Void> - implements OneInputStreamOperator<WriteResult, Void>, BoundedOneInput { + implements OneInputStreamOperator<FlinkWriteResult, Void>, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -96,7 +94,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator<Void> // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List<WriteResult> writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map<Long, List<WriteResult>> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -187,7 +185,7 @@ public void initializeState(StateInitializationContext context) throws Exception // it's safe to assign the max committed checkpoint id from restored flink job to the current // flink job. this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + SinkUtil.getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); NavigableMap<Long, byte[]> uncommittedDataFiles = Maps.newTreeMap(checkpointsState.get().iterator().next()) @@ -212,7 +210,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -220,8 +219,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -280,7 +277,7 @@ private void commitUpToCheckpoint( commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); committerMetrics.updateCommitSummary(summary); pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); } private void commitPendingResult( @@ -303,27 +300,6 @@ private void commitPendingResult( } } - private void deleteCommittedManifests( - List<ManifestFile> manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - private void replacePartitions( NavigableMap<Long, WriteResult> pendingResults, CommitSummary summary, @@ -426,30 +402,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord<WriteResult> element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord<FlinkWriteResult> element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List<WriteResult> writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry<Long, List<WriteResult>> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List<WriteResult> writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); @@ -489,28 +480,4 @@ static ListStateDescriptor<SortedMap<Long, byte[]>> buildStateDescriptor() { longComparator); return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map<String, String> summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java new file mode 100644 index 000000000000..d080169544cd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -0,0 +1,742 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. We will use the + * following: + * + * <ul> + * <li>{@link SupportsPreWriteTopology} which redistributes the data to the writers based on the + * {@link DistributionMode} + * <li>{@link org.apache.flink.api.connector.sink2.SinkWriter} which writes data/delete files, and + * generates the {@link org.apache.iceberg.io.WriteResult} objects for the files + * <li>{@link SupportsPreCommitTopology} which we use to place the {@link + * org.apache.iceberg.flink.sink.IcebergWriteAggregator} which merges the individual {@link + * org.apache.flink.api.connector.sink2.SinkWriter}'s {@link + * org.apache.iceberg.io.WriteResult}s to a single {@link + * org.apache.iceberg.flink.sink.IcebergCommittable} + * <li>{@link org.apache.iceberg.flink.sink.IcebergCommitter} which commits the incoming{@link + * org.apache.iceberg.flink.sink.IcebergCommittable}s to the Iceberg table + * <li>{@link SupportsPostCommitTopology} we could use for incremental compaction later. This is + * not implemented yet. + * </ul> + * + * The job graph looks like below: + * + * <pre>{@code + * Flink sink + * +-----------------------------------------------------------------------------------+ + * | | + * +-------+ | +----------+ +-------------+ +---------------+ | + * | Map 1 | ==> | | writer 1 | | committer 1 | ---> | post commit 1 | | + * +-------+ | +----------+ +-------------+ +---------------+ | + * | \ / \ | + * | \ / \ | + * | \ / \ | + * +-------+ | +----------+ \ +-------------------+ / +-------------+ \ +---------------+ | + * | Map 2 | ==> | | writer 2 | --->| commit aggregator | | committer 2 | | post commit 2 | | + * +-------+ | +----------+ +-------------------+ +-------------+ +---------------+ | + * | Commit only on | + * | committer 1 | + * +-----------------------------------------------------------------------------------+ + * }</pre> + */ +@Experimental +public class IcebergSink + implements Sink<RowData>, + SupportsPreWriteTopology<RowData>, + SupportsCommitter<IcebergCommittable>, + SupportsPreCommitTopology<WriteResult, IcebergCommittable>, + SupportsPostCommitTopology<IcebergCommittable> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + private final TableLoader tableLoader; + private final Map<String, String> snapshotProperties; + private final String uidSuffix; + private final String sinkId; + private final Map<String, String> writeProperties; + private final RowType flinkRowType; + private final SerializableSupplier<Table> tableSupplier; + private final transient FlinkWriteConf flinkWriteConf; + private final List<Integer> equalityFieldIds; + private final boolean upsertMode; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final String branch; + private final boolean overwriteMode; + private final int workerPoolSize; + + private final Table table; + private final List<String> equalityFieldColumns = null; + + private IcebergSink( + TableLoader tableLoader, + Table table, + Map<String, String> snapshotProperties, + String uidSuffix, + Map<String, String> writeProperties, + RowType flinkRowType, + SerializableSupplier<Table> tableSupplier, + FlinkWriteConf flinkWriteConf, + List<Integer> equalityFieldIds, + String branch, + boolean overwriteMode) { + this.tableLoader = tableLoader; + this.snapshotProperties = snapshotProperties; + this.uidSuffix = uidSuffix; + this.writeProperties = writeProperties; + this.flinkRowType = flinkRowType; + this.tableSupplier = tableSupplier; + this.flinkWriteConf = flinkWriteConf; + this.equalityFieldIds = equalityFieldIds; + this.branch = branch; + this.overwriteMode = overwriteMode; + this.table = table; + this.upsertMode = flinkWriteConf.upsertMode(); + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter<RowData> createWriter(InitContext context) { + RowDataTaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + targetDataFileSize, + dataFileFormat, + writeProperties, + equalityFieldIds, + upsertMode); + IcebergStreamWriterMetrics metrics = + new IcebergStreamWriterMetrics(context.metricGroup(), table.name()); + return new IcebergSinkWriter( + tableSupplier.get().name(), + taskWriterFactory, + metrics, + context.getSubtaskId(), + context.getAttemptNumber()); + } + + @Override + public Committer<IcebergCommittable> createCommitter(CommitterInitContext context) { + IcebergFilesCommitterMetrics metrics = + new IcebergFilesCommitterMetrics(context.metricGroup(), table.name()); + return new IcebergCommitter( + tableLoader, branch, snapshotProperties, overwriteMode, workerPoolSize, sinkId, metrics); + } + + @Override + public SimpleVersionedSerializer<IcebergCommittable> getCommittableSerializer() { + return new IcebergCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream<CommittableMessage<IcebergCommittable>> committables) { + // TODO Support small file compaction + } + + @Override + public DataStream<RowData> addPreWriteTopology(DataStream<RowData> inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream<CommittableMessage<IcebergCommittable>> addPreCommitTopology( + DataStream<CommittableMessage<WriteResult>> writeResults) { + TypeInformation<CommittableMessage<IcebergCommittable>> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + String suffix = defaultSuffix(uidSuffix, table.name()); + String preCommitAggregatorUid = String.format("Sink pre-commit aggregator: %s", suffix); + + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + return writeResults + .global() + .transform(preCommitAggregatorUid, typeInformation, new IcebergWriteAggregator(tableLoader)) + .uid(preCommitAggregatorUid) + .setParallelism(1) + .setMaxParallelism(1) + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + .global(); + } + + @Override + public SimpleVersionedSerializer<WriteResult> getWriteResultSerializer() { + return new WriteResultSerializer(); + } + + public static class Builder { + private TableLoader tableLoader; + private String uidSuffix = ""; + private Function<String, DataStream<RowData>> inputCreator = null; + private TableSchema tableSchema; + private SerializableTable table; + private final Map<String, String> writeOptions = Maps.newHashMap(); + private final Map<String, String> snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private List<String> equalityFieldColumns = null; + + private Builder() {} + + private Builder forRowData(DataStream<RowData> newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forRow(DataStream<Row> input, TableSchema inputTableSchema) { + RowType rowType = (RowType) inputTableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = inputTableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return forMapperOutputType( + input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(inputTableSchema); + } + + private <T> Builder forMapperOutputType( + DataStream<T> input, MapFunction<T, RowData> mapper, TypeInformation<RowData> outputType) { + this.inputCreator = + newUidSuffix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism of map operator same as its input to keep map operator + // chaining its input, and avoid rebalanced by default. + SingleOutputStreamOperator<RowData> inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidSuffix != null) { + String uid = String.format("Sink pre-writer mapper: %s", newUidSuffix); + inputStream.name(uid).uid(uid); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link SerializableTable} instance is used for initializing {@link + * IcebergStreamWriter} which will write all the records into {@link DataFile}s and emit them to + * downstream operator. Providing a table would avoid so many table loading from each separate + * task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = (SerializableTable) SerializableTable.copyOf(newTable); + return this; + } + + /** + * The table loader is used for loading tables in {@link + * org.apache.iceberg.flink.sink.IcebergCommitter} lazily, we need this loader because {@link + * Table} is not serializable and could not just use the loaded table from Builder#table in the + * remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + TableLoader tableLoader() { + return tableLoader; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map<String, String> properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the IcebergSink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List<String> columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid suffix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator). Actual operator uid will be appended + * with a suffix like "Sink Committer: $uidSuffix". + * + * <p>Flink auto generates operator uid if not set explicitly. It is a recommended <a + * href="https://ci.apache.org/projects/flink/flink-docs-master/docs/ops/production_ready/"> + * best-practice to set uid for all operators</a> before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid. + * + * <p>Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newSuffix suffix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidSuffix(String newSuffix) { + this.uidSuffix = newSuffix; + return this; + } + + public Builder snapshotProperties(Map<String, String> properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + IcebergSink build() { + + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader(), "Table loader shouldn't be null"); + + // Set the table if it is not yet set in the builder, so we can do the equalityId checks + SerializableTable serializableTable = checkAndGetTable(tableLoader(), table); + this.table = serializableTable; + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + SerializableSupplier<Table> tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List<Integer> equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink<RowData> append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream<RowData> rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: <uidSuffix> + DataStreamSink<RowData> rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map<String, String> writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map<String, String> writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream<RowData> distributeDataStream(DataStream<RowData> input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param <T> the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static <T> Builder builderFor( + DataStream<T> input, MapFunction<T, RowData> mapper, TypeInformation<RowData> outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream<RowData> input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter<RowData, WriteResult> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory<RowData> taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter<RowData> writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory<RowData> taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection<WriteResult> prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..412d6c7081bf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter<T> extends AbstractStreamOperator<WriteResult> - implements OneInputStreamOperator<T, WriteResult>, BoundedOneInput { +class IcebergStreamWriter<T> extends AbstractStreamOperator<FlinkWriteResult> + implements OneInputStreamOperator<T, FlinkWriteResult>, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory<T> taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,20 +90,20 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator<CommittableMessage<IcebergCommittable>> + implements OneInputStreamOperator< + CommittableMessage<WriteResult>, CommittableMessage<IcebergCommittable>> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection<WriteResult> results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage<IcebergCommittable> summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage<IcebergCommittable> message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection<WriteResult> writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord<CommittableMessage<WriteResult>> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage<WriteResult>) element.getValue()).getCommittable()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..1cb7f4dea1e8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -26,13 +26,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier<Table> tableSupplier; private final Map<String, String> props; private final String flinkJobId; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String OPERATOR_ID = "flink.operator-id"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List<Integer> checkAndGetEqualityFieldIds(Table table, List<String> equalityFieldColumns) { + List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map<String, String> summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +class WriteResultSerializer implements SimpleVersionedSerializer<WriteResult> { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 52d8a2f16f99..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 @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Comparator; import java.util.Map; import java.util.NavigableMap; import java.util.Set; @@ -29,8 +28,6 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -38,9 +35,8 @@ import org.slf4j.LoggerFactory; /** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. */ class AggregatedStatisticsTracker { private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); @@ -51,10 +47,9 @@ class AggregatedStatisticsTracker { private final int downstreamParallelism; private final StatisticsType statisticsType; private final int switchToSketchThreshold; - private final Comparator<StructLike> comparator; private final NavigableMap<Long, Aggregation> aggregationsPerCheckpoint; - private AggregatedStatistics completedStatistics; + private CompletedStatistics completedStatistics; AggregatedStatisticsTracker( String operatorName, @@ -64,7 +59,7 @@ class AggregatedStatisticsTracker { int downstreamParallelism, StatisticsType statisticsType, int switchToSketchThreshold, - @Nullable AggregatedStatistics restoredStatistics) { + @Nullable CompletedStatistics restoredStatistics) { this.operatorName = operatorName; this.parallelism = parallelism; this.statisticsSerializer = @@ -74,11 +69,10 @@ class AggregatedStatisticsTracker { this.switchToSketchThreshold = switchToSketchThreshold; this.completedStatistics = restoredStatistics; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); this.aggregationsPerCheckpoint = Maps.newTreeMap(); } - AggregatedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { long checkpointId = event.checkpointId(); LOG.debug( "Handling statistics event from subtask {} of operator {} for checkpoint {}", @@ -105,7 +99,6 @@ AggregatedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event parallelism, downstreamParallelism, switchToSketchThreshold, - comparator, statisticsType, StatisticsUtil.collectType(statisticsType, completedStatistics))); DataStatistics dataStatistics = @@ -140,7 +133,6 @@ static class Aggregation { private final int parallelism; private final int downstreamParallelism; private final int switchToSketchThreshold; - private final Comparator<StructLike> comparator; private final StatisticsType configuredType; private StatisticsType currentType; private Map<SortKey, Long> mapStatistics; @@ -150,14 +142,12 @@ static class Aggregation { int parallelism, int downstreamParallelism, int switchToSketchThreshold, - Comparator<StructLike> comparator, StatisticsType configuredType, StatisticsType currentType) { this.subtaskSet = Sets.newHashSet(); this.parallelism = parallelism; this.downstreamParallelism = downstreamParallelism; this.switchToSketchThreshold = switchToSketchThreshold; - this.comparator = comparator; this.configuredType = configuredType; this.currentType = currentType; @@ -233,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -246,20 +238,24 @@ private void convertCoordinatorToSketch() { this.mapStatistics = null; } - private AggregatedStatistics completedStatistics(long checkpointId) { + private CompletedStatistics completedStatistics(long checkpointId) { if (currentType == StatisticsType.Map) { LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); - return AggregatedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch<SortKey> sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return AggregatedStatistics.fromRangeBounds( - checkpointId, SketchUtil.rangeBounds(downstreamParallelism, comparator, sketch)); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java new file mode 100644 index 000000000000..e4cba174f0f2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). + */ +class CompletedStatistics { + private final long checkpointId; + private final StatisticsType type; + private final Map<SortKey, Long> keyFrequency; + private final SortKey[] keySamples; + + static CompletedStatistics fromKeyFrequency(long checkpointId, Map<SortKey, Long> stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( + long checkpointId, + StatisticsType type, + Map<SortKey, Long> keyFrequency, + SortKey[] keySamples) { + this.checkpointId = checkpointId; + this.type = type; + this.keyFrequency = keyFrequency; + this.keySamples = keySamples; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("keyFrequency", keyFrequency) + .add("keySamples", keySamples) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof CompletedStatistics)) { + return false; + } + + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(keyFrequency, other.keyFrequency()) + && Arrays.equals(keySamples, other.keySamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + Map<SortKey, Long> keyFrequency() { + return keyFrequency; + } + + SortKey[] keySamples() { + return keySamples; + } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java similarity index 62% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java index 56ba5e04f05a..1ac0e386a011 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -35,17 +35,17 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.iceberg.SortKey; -public class AggregatedStatisticsSerializer extends TypeSerializer<AggregatedStatistics> { +class CompletedStatisticsSerializer extends TypeSerializer<CompletedStatistics> { private final TypeSerializer<SortKey> sortKeySerializer; private final EnumSerializer<StatisticsType> statisticsTypeSerializer; private final MapSerializer<SortKey, Long> keyFrequencySerializer; - private final ListSerializer<SortKey> rangeBoundsSerializer; + private final ListSerializer<SortKey> keySamplesSerializer; - AggregatedStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + CompletedStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { this.sortKeySerializer = sortKeySerializer; this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); - this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); } @Override @@ -54,23 +54,23 @@ public boolean isImmutableType() { } @Override - public TypeSerializer<AggregatedStatistics> duplicate() { - return new AggregatedStatisticsSerializer(sortKeySerializer); + public TypeSerializer<CompletedStatistics> duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); } @Override - public AggregatedStatistics createInstance() { - return new AggregatedStatistics(0, StatisticsType.Map, Collections.emptyMap(), null); + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); } @Override - public AggregatedStatistics copy(AggregatedStatistics from) { - return new AggregatedStatistics( - from.checkpointId(), from.type(), from.keyFrequency(), from.rangeBounds()); + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); } @Override - public AggregatedStatistics copy(AggregatedStatistics from, AggregatedStatistics reuse) { + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { // no benefit of reuse return copy(from); } @@ -81,35 +81,33 @@ public int getLength() { } @Override - public void serialize(AggregatedStatistics record, DataOutputView target) throws IOException { + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { target.writeLong(record.checkpointId()); statisticsTypeSerializer.serialize(record.type(), target); if (record.type() == StatisticsType.Map) { keyFrequencySerializer.serialize(record.keyFrequency(), target); } else { - rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); } } @Override - public AggregatedStatistics deserialize(DataInputView source) throws IOException { + public CompletedStatistics deserialize(DataInputView source) throws IOException { long checkpointId = source.readLong(); StatisticsType type = statisticsTypeSerializer.deserialize(source); - Map<SortKey, Long> keyFrequency = null; - SortKey[] rangeBounds = null; if (type == StatisticsType.Map) { - keyFrequency = keyFrequencySerializer.deserialize(source); + Map<SortKey, Long> keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); } else { - List<SortKey> sortKeys = rangeBoundsSerializer.deserialize(source); - rangeBounds = new SortKey[sortKeys.size()]; - rangeBounds = sortKeys.toArray(rangeBounds); + List<SortKey> sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); } - - return new AggregatedStatistics(checkpointId, type, keyFrequency, rangeBounds); } @Override - public AggregatedStatistics deserialize(AggregatedStatistics reuse, DataInputView source) + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) throws IOException { // not much benefit to reuse return deserialize(source); @@ -122,11 +120,15 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof AggregatedStatisticsSerializer)) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { return false; } - AggregatedStatisticsSerializer other = (AggregatedStatisticsSerializer) obj; + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; return Objects.equals(sortKeySerializer, other.sortKeySerializer); } @@ -136,21 +138,20 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot<AggregatedStatistics> snapshotConfiguration() { - return new AggregatedStatisticsSerializerSnapshot(this); + public TypeSerializerSnapshot<CompletedStatistics> snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); } - public static class AggregatedStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - AggregatedStatistics, AggregatedStatisticsSerializer> { + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<CompletedStatistics, CompletedStatisticsSerializer> { private static final int CURRENT_VERSION = 1; /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public AggregatedStatisticsSerializerSnapshot() {} + public CompletedStatisticsSerializerSnapshot() {} @SuppressWarnings("checkstyle:RedundantModifier") - public AggregatedStatisticsSerializerSnapshot(AggregatedStatisticsSerializer serializer) { + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { super(serializer); } @@ -161,15 +162,15 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer<?>[] getNestedSerializers( - AggregatedStatisticsSerializer outerSerializer) { + CompletedStatisticsSerializer outerSerializer) { return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; } @Override - protected AggregatedStatisticsSerializer createOuterSerializerWithNestedSerializers( + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer<?>[] nestedSerializers) { SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; - return new AggregatedStatisticsSerializer(sortKeySerializer); + return new CompletedStatisticsSerializer(sortKeySerializer); } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 6e9e4f6fa6c2..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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -37,9 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -60,17 +63,21 @@ class DataStatisticsCoordinator implements OperatorCoordinator { private final OperatorCoordinator.Context context; private final Schema schema; private final SortOrder sortOrder; + private final Comparator<StructLike> comparator; private final int downstreamParallelism; private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; private final ExecutorService coordinatorExecutor; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer<AggregatedStatistics> aggregatedStatisticsSerializer; + private final TypeSerializer<CompletedStatistics> completedStatisticsSerializer; + private final TypeSerializer<GlobalStatistics> globalStatisticsSerializer; private transient boolean started; private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private transient AggregatedStatistics completedStatistics; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; DataStatisticsCoordinator( String operatorName, @@ -78,13 +85,16 @@ class DataStatisticsCoordinator implements OperatorCoordinator { Schema schema, SortOrder sortOrder, int downstreamParallelism, - StatisticsType statisticsType) { + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { this.operatorName = operatorName; this.context = context; this.schema = schema; this.sortOrder = sortOrder; + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( @@ -92,13 +102,16 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); - this.aggregatedStatisticsSerializer = new AggregatedStatisticsSerializer(sortKeySerializer); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called this.aggregatedStatisticsTracker = new AggregatedStatisticsTracker( operatorName, @@ -185,32 +198,104 @@ private void ensureStarted() { } private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = + CompletedStatistics maybeCompletedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendAggregatedStatisticsToSubtasks(completedStatistics.checkpointId(), completedStatistics); + if (maybeCompletedStatistics != null) { + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator<StructLike> comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendAggregatedStatisticsToSubtasks( - long checkpointId, AggregatedStatistics globalStatistics) { - callInCoordinatorThread( + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( () -> { + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary StatisticsEvent statisticsEvent = - StatisticsEvent.createAggregatedStatisticsEvent( - checkpointId, globalStatistics, aggregatedStatisticsSerializer); + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } - - return null; }, String.format( "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } } @Override @@ -223,8 +308,14 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof StatisticsEvent); - handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -239,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeAggregatedStatistics( - completedStatistics, aggregatedStatisticsSerializer)); + 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)); } @@ -253,8 +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, @@ -265,8 +360,12 @@ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); this.completedStatistics = - StatisticsUtil.deserializeAggregatedStatistics( - checkpointData, aggregatedStatisticsSerializer); + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); } @Override @@ -290,7 +389,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", subtask, attemptNumber, operatorName); @@ -316,10 +415,15 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics completedStatistics() { + CompletedStatistics completedStatistics() { return completedStatistics; } + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + private static class SubtaskGateways { private final String operatorName; private final Map<Integer, SubtaskGateway>[] gateways; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java index ffb428283785..9d7d989c298e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -37,6 +37,7 @@ public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCo private final SortOrder sortOrder; private final int downstreamParallelism; private final StatisticsType type; + private final double closeFileCostWeightPercentage; public DataStatisticsCoordinatorProvider( String operatorName, @@ -44,18 +45,26 @@ public DataStatisticsCoordinatorProvider( Schema schema, SortOrder sortOrder, int downstreamParallelism, - StatisticsType type) { + StatisticsType type, + double closeFileCostWeightPercentage) { super(operatorID); this.operatorName = operatorName; this.schema = schema; this.sortOrder = sortOrder; this.downstreamParallelism = downstreamParallelism; this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { return new DataStatisticsCoordinator( - operatorName, context, schema, sortOrder, downstreamParallelism, type); + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 2910471762cd..7995a8a5b181 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -60,16 +60,16 @@ public class DataStatisticsOperator extends AbstractStreamOperator<StatisticsOrR private final int downstreamParallelism; private final StatisticsType statisticsType; private final TypeSerializer<DataStatistics> taskStatisticsSerializer; - private final TypeSerializer<AggregatedStatistics> aggregatedStatisticsSerializer; + private final TypeSerializer<GlobalStatistics> globalStatisticsSerializer; private transient int parallelism; private transient int subtaskIndex; - private transient ListState<AggregatedStatistics> globalStatisticsState; + private transient ListState<GlobalStatistics> globalStatisticsState; // current statistics type may be different from the config due to possible // migration from Map statistics to Sketch statistics when high cardinality detected private transient volatile StatisticsType taskStatisticsType; private transient volatile DataStatistics localStatistics; - private transient volatile AggregatedStatistics globalStatistics; + private transient volatile GlobalStatistics globalStatistics; DataStatisticsOperator( String operatorName, @@ -87,31 +87,57 @@ public class DataStatisticsOperator extends AbstractStreamOperator<StatisticsOrR SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); - this.aggregatedStatisticsSerializer = new AggregatedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void initializeState(StateInitializationContext context) throws Exception { this.parallelism = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); this.subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + + // Use union state so that new subtasks can also restore global statistics during scale-up. this.globalStatisticsState = context .getOperatorStateStore() .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", aggregatedStatisticsSerializer)); + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); if (context.isRestored()) { if (globalStatisticsState.get() == null || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( + LOG.info( "Operator {} subtask {} doesn't have global statistics state to restore", operatorName, subtaskIndex); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); LOG.info( - "Operator {} subtask {} restoring global statistics state", operatorName, subtaskIndex); - this.globalStatistics = globalStatisticsState.get().iterator().next(); + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; } + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); } this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); @@ -139,14 +165,16 @@ public void handleOperatorEvent(OperatorEvent event) { operatorName, subtaskIndex, statisticsEvent.checkpointId()); - globalStatistics = - StatisticsUtil.deserializeAggregatedStatistics( - statisticsEvent.statisticsBytes(), aggregatedStatisticsSerializer); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); checkStatisticsTypeMigration(); - output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } } - @SuppressWarnings("unchecked") @Override public void processElement(StreamRecord<RowData> streamRecord) { // collect data statistics @@ -204,6 +232,7 @@ && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); } + @SuppressWarnings("unchecked") private void checkStatisticsTypeMigration() { // only check if the statisticsType config is Auto and localStatistics is currently Map type if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { @@ -231,7 +260,7 @@ DataStatistics localStatistics() { } @VisibleForTesting - AggregatedStatistics globalStatistics() { + GlobalStatistics globalStatistics() { return globalStatistics; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory<StatisticsOrRecord> + implements CoordinatedOperatorFactory<StatisticsOrRecord>, + OneInputStreamOperatorFactory<RowData, StatisticsOrRecord> { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public <T extends StreamOperator<StatisticsOrRecord>> T createStreamOperator( + StreamOperatorParameters<StatisticsOrRecord> parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..a7fe2b30b865 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer<GlobalStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final ListSerializer<SortKey> rangeBoundsSerializer; + private final ListSerializer<Integer> intsSerializer; + private final ListSerializer<Long> longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<GlobalStatistics> duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry<SortKey, KeyAssignment> entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map<SortKey, KeyAssignment> keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List<Integer> assignedSubtasks = intsSerializer.deserialize(source); + List<Long> subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List<SortKey> sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<GlobalStatistics> snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<GlobalStatistics, GlobalStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..781bcc646023 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List<Integer> assignedSubtasks; + private final List<Long> subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List<Integer> assignedSubtasks, + List<Long> subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List<Integer> assignedSubtasks() { + return assignedSubtasks; + } + + List<Long> subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** + * Select a subtask for the key. + * + * @return subtask id + */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..9d8167460a1b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map<SortKey, KeyAssignment> keyAssignments; + + MapAssignment(int numPartitions, Map<SortKey, KeyAssignment> keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map<SortKey, Long> mapStatistics, + double closeFileCostWeightPercentage, + Comparator<StructLike> comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map<SortKey, KeyAssignment> keyAssignments() { + return keyAssignments; + } + + /** + * Returns assignment summary for every subtask. + * + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map<Integer, Pair<Long, Integer>> assignmentInfo() { + Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map<SortKey, KeyAssignment> assignment( + int numPartitions, + Map<SortKey, Long> mapStatistics, + double closeFileCostWeightPercentage, + Comparator<StructLike> comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map<SortKey, KeyAssignment> buildAssignment( + int numPartitions, + NavigableMap<SortKey, Long> sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map<SortKey, KeyAssignment> assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List<Integer> assignedSubtasks = Lists.newArrayList(); + List<Long> subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index 298426cee872..f36a078c94e0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -18,29 +18,14 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,47 +46,28 @@ class MapRangePartitioner implements Partitioner<RowData> { private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; - private final Comparator<StructLike> comparator; - private final Map<SortKey, Long> mapStatistics; - private final double closeFileCostInWeightPercentage; + private final MapAssignment mapAssignment; // Counter that tracks how many times a new key encountered // where there is no traffic statistics learned about it. private long newSortKeyCounter; private long lastNewSortKeyLogTimeMilli; - // lazily computed due to the need of numPartitions - private Map<SortKey, KeyAssignment> assignment; - private NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - Map<SortKey, Long> mapStatistics, - double closeFileCostInWeightPercentage) { - mapStatistics.forEach( - (key, value) -> - Preconditions.checkArgument( - value > 0, "Invalid statistics: weight is 0 for key %s", key)); - + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = mapStatistics; - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.mapAssignment = mapAssignment; this.newSortKeyCounter = 0; this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); } @Override public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map<SortKey, KeyAssignment> assignmentMap = assignment(numPartitions); // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; if (keyAssignment == null) { LOG.trace( "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", @@ -118,268 +84,12 @@ public int partition(RowData row, int numPartitions) { lastNewSortKeyLogTimeMilli = now; newSortKeyCounter = 0; } - return (int) (newSortKeyCounter % numPartitions); + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); } - return keyAssignment.select(); - } - - @VisibleForTesting - Map<SortKey, KeyAssignment> assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map<SortKey, Long> mapStatistics() { - return mapStatistics; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map<Integer, Pair<Long, Integer>> assignmentInfo() { - Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map<SortKey, KeyAssignment> buildAssignment( - int numPartitions, - NavigableMap<SortKey, Long> sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map<SortKey, KeyAssignment> assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List<Integer> assignedSubtasks = Lists.newArrayList(); - List<Long> subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List<Integer> assignedSubtasks, - List<Long> subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java new file mode 100644 index 000000000000..6608b938f5a8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ +@Internal +public class RangePartitioner implements Partitioner<StatisticsOrRecord> { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner<RowData> delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner<RowData> delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java new file mode 100644 index 000000000000..dddb0d8722c0 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; + +class SketchRangePartitioner implements Partitioner<RowData> { + private final SortKey sortKey; + private final Comparator<StructLike> comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index bfd3082a5aa7..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 @@ -22,7 +22,6 @@ import java.util.Comparator; import java.util.Map; import java.util.function.Consumer; -import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.iceberg.SortKey; import org.apache.iceberg.StructLike; @@ -90,37 +89,32 @@ static int determineOperatorReservoirSize(int operatorParallelism, int numPartit /** * To understand how range bounds are used in range partitioning, here is an example for human * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be - * <li>age <= 15 - * <li>age > 15 && age <= 32 - * <li>age >32 && age <= 60 - * <li>age > 60 + * + * <ul> + * <li>age <= 15 + * <li>age > 15 && age <= 32 + * <li>age >32 && age <= 60 + * <li>age > 60 + * </ul> + * + * <p>Assumption is that a single key is not dominant enough to span multiple subtasks. * * @param numPartitions number of partitions which maps to downstream operator parallelism - * @param sketch aggregated reservoir sampling sketch - * @return list of range partition bounds. It should be a sorted list (ascending). Number of items - * should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list */ static SortKey[] rangeBounds( - int numPartitions, Comparator<StructLike> comparator, ReservoirItemsSketch<SortKey> sketch) { - SortKey[] sortKeys = sketch.getSamples(); - return determineBounds(Math.min(numPartitions, sortKeys.length), comparator, sortKeys); - } - - /** - * This assumes the sort keys have equal weight, which is usually the case for high-cardinality - * scenarios (like device_id, user_id, uuid etc.). - */ - static SortKey[] determineBounds( - int numPartitions, Comparator<StructLike> comparator, SortKey[] sortKeys) { + int numPartitions, Comparator<StructLike> comparator, SortKey[] samples) { // sort the keys first - Arrays.sort(sortKeys, comparator); + Arrays.sort(samples, comparator); int numCandidates = numPartitions - 1; SortKey[] candidates = new SortKey[numCandidates]; - int step = (int) Math.ceil((double) sortKeys.length / numPartitions); + int step = (int) Math.ceil((double) samples.length / numPartitions); int position = step - 1; int numChosen = 0; - while (position < sortKeys.length && numChosen < numCandidates) { - SortKey candidate = sortKeys[position]; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; // skip duplicate values if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { // linear probe for the next distinct value @@ -145,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator<StructLike> comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 9c0e65a1fc49..b82fc8250763 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -24,8 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; -import org.apache.flink.annotation.VisibleForTesting; +import java.util.stream.Collectors; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -41,6 +42,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer<SortKey> { @@ -319,12 +321,25 @@ public TypeSerializerSchemaCompatibility<SortKey> resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } + // Sort order should be identical SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(oldSnapshot.schema, schema); + Set<Integer> sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema oldSortSchema = TypeUtil.project(oldSnapshot.schema, sortFieldIds); + + List<String> compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, oldSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override @@ -340,17 +355,5 @@ private void readV1(DataInputView in) throws IOException { this.schema = SchemaParser.fromJson(schemaJson); this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); } - - @VisibleForTesting - static <T> TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List<String> compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } } } 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<SortField> sortFields = sortOrder.fields(); + int size = sortFields.size(); + List<Types.NestedField> transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index f3391d72297f..f6fcdb8b16ef 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -32,27 +32,34 @@ class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; + private final boolean applyImmediately; - private StatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; } static StatisticsEvent createTaskStatisticsEvent( long checkpointId, DataStatistics statistics, TypeSerializer<DataStatistics> statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. return new StatisticsEvent( - checkpointId, StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer)); + checkpointId, + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); } - static StatisticsEvent createAggregatedStatisticsEvent( - long checkpointId, - AggregatedStatistics statistics, - TypeSerializer<AggregatedStatistics> statisticsSerializer) { + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer<GlobalStatistics> statisticsSerializer, + boolean applyImmediately) { return new StatisticsEvent( - checkpointId, - StatisticsUtil.serializeAggregatedStatistics(statistics, statisticsSerializer)); + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); } long checkpointId() { @@ -62,4 +69,8 @@ long checkpointId() { byte[] statisticsBytes() { return statisticsBytes; } + + boolean applyImmediately() { + return applyImmediately; + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index c251ba1360fc..bc28df2b0e22 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -40,10 +40,10 @@ public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private AggregatedStatistics statistics; + private GlobalStatistics statistics; private RowData record; - private StatisticsOrRecord(AggregatedStatistics statistics, RowData record) { + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; @@ -54,7 +54,7 @@ static StatisticsOrRecord fromRecord(RowData record) { return new StatisticsOrRecord(null, record); } - static StatisticsOrRecord fromStatistics(AggregatedStatistics statistics) { + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { return new StatisticsOrRecord(statistics, null); } @@ -69,7 +69,7 @@ static StatisticsOrRecord reuseRecord( } static StatisticsOrRecord reuseStatistics( - StatisticsOrRecord reuse, TypeSerializer<AggregatedStatistics> statisticsSerializer) { + StatisticsOrRecord reuse, TypeSerializer<GlobalStatistics> statisticsSerializer) { if (reuse.hasStatistics()) { return reuse; } else { @@ -86,11 +86,11 @@ public boolean hasRecord() { return record != null; } - AggregatedStatistics statistics() { + GlobalStatistics statistics() { return statistics; } - void statistics(AggregatedStatistics newStatistics) { + void statistics(GlobalStatistics newStatistics) { this.statistics = newStatistics; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index 7e690f61a58b..d4ae2b359679 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -30,11 +30,11 @@ @Internal class StatisticsOrRecordSerializer extends TypeSerializer<StatisticsOrRecord> { - private final TypeSerializer<AggregatedStatistics> statisticsSerializer; + private final TypeSerializer<GlobalStatistics> statisticsSerializer; private final TypeSerializer<RowData> recordSerializer; StatisticsOrRecordSerializer( - TypeSerializer<AggregatedStatistics> statisticsSerializer, + TypeSerializer<GlobalStatistics> statisticsSerializer, TypeSerializer<RowData> recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,7 +48,7 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override public TypeSerializer<StatisticsOrRecord> duplicate() { - TypeSerializer<AggregatedStatistics> duplicateStatisticsSerializer = + TypeSerializer<GlobalStatistics> duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) @@ -84,8 +84,7 @@ public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse to.record(record); } else { to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - AggregatedStatistics statistics = - statisticsSerializer.copy(from.statistics(), to.statistics()); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); to.statistics(statistics); } @@ -130,7 +129,7 @@ public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView so to.record(record); } else { to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - AggregatedStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); to.statistics(statistics); } @@ -200,8 +199,8 @@ protected TypeSerializer<?>[] getNestedSerializers( @Override protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer<?>[] nestedSerializers) { - TypeSerializer<AggregatedStatistics> statisticsSerializer = - (TypeSerializer<AggregatedStatistics>) nestedSerializers[0]; + TypeSerializer<GlobalStatistics> statisticsSerializer = + (TypeSerializer<GlobalStatistics>) nestedSerializers[0]; TypeSerializer<RowData> recordSerializer = (TypeSerializer<RowData>) nestedSerializers[1]; return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java index 9d3d128535fe..5d48ec57ca49 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -60,20 +60,41 @@ static DataStatistics deserializeDataStatistics( } } - static byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer<AggregatedStatistics> statisticsSerializer) { + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer<CompletedStatistics> statisticsSerializer) { try { DataOutputSerializer out = new DataOutputSerializer(1024); - statisticsSerializer.serialize(aggregatedStatistics, out); + statisticsSerializer.serialize(completedStatistics, out); return out.getCopyOfBuffer(); } catch (IOException e) { throw new UncheckedIOException("Fail to serialize aggregated statistics", e); } } - static AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer<AggregatedStatistics> statisticsSerializer) { + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer<CompletedStatistics> statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer<GlobalStatistics> statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer<GlobalStatistics> statisticsSerializer) { try { DataInputDeserializer input = new DataInputDeserializer(bytes); return statisticsSerializer.deserialize(input); @@ -86,10 +107,18 @@ static StatisticsType collectType(StatisticsType config) { return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; } + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + static StatisticsType collectType( - StatisticsType config, @Nullable AggregatedStatistics restoredStatistics) { - if (restoredStatistics != null) { - return restoredStatistics.type(); + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); } return collectType(config); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..5718f4b93825 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -28,6 +28,8 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +39,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -61,10 +66,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -72,6 +79,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; @@ -95,6 +103,11 @@ public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEn private final SerializableRecordEmitter<T> emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List<IcebergSourceSplit> batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -130,16 +143,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List<IcebergSourceSplit> planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List<IcebergSourceSplit> splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -205,26 +228,69 @@ private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumer // Only do scan planning if nothing is restored from checkpoint state List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static <T> Builder<T> builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder<RowData> forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param <T> output type + * @return an IcebergSource builder + */ + public static <T> Builder<T> forOutputType(RowDataConverter<T> converter) { + return new Builder<T>().converter(converter); + } + public static class Builder<T> { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator<IcebergSourceSplit> splitComparator; private ReaderFunction<T> readerFunction; + private RowDataConverter<T> converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +321,28 @@ public Builder<T> splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder<T> readerFunction(ReaderFunction<T> newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder<T> converter(RowDataConverter<T> newConverter) { + this.converter = newConverter; + return this; + } + public Builder<T> flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -482,6 +565,10 @@ public IcebergSource<T> build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); @@ -506,24 +593,7 @@ public IcebergSource<T> build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters()); - this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -544,5 +614,75 @@ public IcebergSource<T> build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream<T> buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource<T> source = build(); + TypeInformation<T> outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource<T> stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static <T> TypeInformation<T> outputTypeInfo( + RowDataConverter<T> converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation<T>) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + + private ReaderFunction<T> readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction<T>) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction<T>) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource<RowData> createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream<RowData> createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource<RowData> source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 6c9a855bc149..280a126a46ce 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -37,7 +38,8 @@ import org.slf4j.LoggerFactory; abstract class AbstractIcebergEnumerator - implements SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> { + implements SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState>, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext<IcebergSourceSplit> 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<IcebergSourceSplit> splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter<GenericRecord> { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation<GenericRecord> outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation<GenericRecord> getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction<GenericRecord> { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction<T> extends DataIteratorReaderFunction<T> { + private final RowDataConverter<T> converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List<Expression> filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter<T> converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List<Expression> filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator<T> createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader<T> implements FileScanTaskReader<T> { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter<T> converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter<T> converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator<T> open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator<T> extends DataIterator<T> { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader<T> fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param <T> output type + */ +public interface RowDataConverter<T> + extends Function<RowData, T>, ResultTypeQueryable<T>, Serializable {} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction<RowData> { private final FileIO io; private final EncryptionManager encryption; private final List<Expression> 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<Expression> 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<Expression> 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<RowData> createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/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<String, String> props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry<String, String> 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/HadoopCatalogResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/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/MiniFlinkClusterExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java index 9a73b80e077d..d2e086aa448e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; @@ -50,4 +51,17 @@ public static MiniClusterExtension createWithClassloaderCheckDisabled() { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ce6caca12158..1767f774922a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Duration; @@ -71,7 +72,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; -import org.junit.Assert; public class SimpleDataUtil { @@ -268,13 +268,13 @@ public static boolean equalsRecords(List<Record> expected, List<Record> actual, } public static void assertRecordsEqual(List<Record> expected, List<Record> actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); + assertThat(actual).hasSameSizeAs(expected); Types.StructType type = schema.asStruct(); StructLikeSet expectedSet = StructLikeSet.create(type); expectedSet.addAll(expected); StructLikeSet actualSet = StructLikeSet.create(type); actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } /** @@ -297,7 +297,7 @@ public static void assertTableRecords(Table table, List<Record> expected, String Snapshot snapshot = latestSnapshot(table, branch); if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); + assertThat(expected).isEmpty(); return; } @@ -313,7 +313,7 @@ public static void assertTableRecords(Table table, List<Record> expected, String actualSet.add(record); } - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/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<Row> sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator<Row> iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable<Row> expected, Iterable<Row> actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable<Row> expected, Iterable<Row> 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<String, String> props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry<String, String> 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/TestFixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java index 884ea2d1d3b1..b9a7d5b1d589 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -46,6 +46,8 @@ private TestFixtures() {} public static final String SINK_TABLE = "t_sink"; public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); public static final Schema TS_SCHEMA = new Schema( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/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<Row> 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<String, String> 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<Row> 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<Long, List<DataFile>> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List<DataFile> 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..b63547d433a4 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List<Object[]> 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<String, String> 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<Row> 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<Row> 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<String, String> 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<Long, List<DataFile>> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List<DataFile> dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List<List<Row>> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List<Row> charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List<Row> flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map<String, String> tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/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<String, String> 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/ConstantsForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java new file mode 100644 index 000000000000..36e162d4f068 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +class ConstantsForTests { + public static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + static final String DUMMY_NAME = "dummy"; + + private ConstantsForTests() { + // Do not instantiate + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java new file mode 100644 index 000000000000..7a523035b7fb --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class MetricsReporterFactoryForTests implements MetricReporterFactory { + private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); + private static final Pattern FULL_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static Map<String, Counter> counters = Maps.newConcurrentMap(); + private static Map<String, Gauge> gauges = Maps.newConcurrentMap(); + private static Set<String> monitoredMetricNames; + + public MetricsReporterFactoryForTests() { + monitoredMetricNames = + Arrays.stream(TableMaintenanceMetrics.class.getDeclaredFields()) + .map( + f -> { + try { + return f.get(null).toString(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + } + + @Override + public MetricReporter createMetricReporter(Properties properties) { + return INSTANCE; + } + + public static void reset() { + counters = Maps.newConcurrentMap(); + gauges = Maps.newConcurrentMap(); + } + + public static Long counter(String name) { + return counterValues().get(name); + } + + public static Long gauge(String name) { + return gaugeValues().get(name); + } + + public static void assertGauges(Map<String, Long> expected) { + assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + } + + public static void assertCounters(Map<String, Long> expected) { + assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + } + + private static Map<String, Long> gaugeValues() { + return gauges.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> (Long) entry.getValue().getValue())); + } + + private static Map<String, Long> counterValues() { + return counters.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> entry.getValue().getCount())); + } + + private static Map<String, Long> filter(Map<String, Long> original, Map<String, Long> filter) { + return original.entrySet().stream() + .filter( + entry -> { + Long filterValue = filter.get(entry.getKey()); + return filterValue == null || filterValue != -1; + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static String longName(String fullName) { + Matcher matcher = FULL_METRIC_NAME.matcher(fullName); + if (!matcher.matches()) { + throw new RuntimeException(String.format("Can't parse simplified metrics name %s", fullName)); + } + + return matcher.group(1) + "." + matcher.group(2) + "." + matcher.group(3); + } + + private static class TestMetricsReporter implements MetricReporter { + @Override + public void open(MetricConfig config) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + if (monitoredMetricNames.contains(metricName)) { + if (metric instanceof Counter) { + counters.put(group.getMetricIdentifier(metricName), (Counter) metric); + } + + if (metric instanceof Gauge) { + gauges.put(group.getMetricIdentifier(metricName), (Gauge) metric); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + // do nothing + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java index 269ae681b02e..225853086545 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -18,28 +18,36 @@ */ 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 java.io.File; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.iceberg.flink.FlinkCatalogFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; class OperatorTestBase { private static final int NUMBER_TASK_MANAGERS = 1; private static final int SLOTS_PER_TASK_MANAGER = 8; + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); static final String TABLE_NAME = "test_table"; @RegisterExtension - protected static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + 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)) + .setConfiguration(config()) .build()); @RegisterExtension @@ -48,4 +56,101 @@ class OperatorTestBase { "catalog", ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), "db"); + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + + protected static TriggerLockFactory lockFactory() { + return new TriggerLockFactory() { + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + }; + } + + /** + * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it + * stops the job with a savepoint. + * + * @param jobClient the job to close + * @param savepointDir the savepointDir to store the last savepoint. If <code>null</code> then + * stop without a savepoint. + * @return configuration for restarting the job from the savepoint + */ + public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + Configuration conf = new Configuration(); + if (jobClient != null) { + if (savepointDir != null) { + // Stop with savepoint + jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); + // Wait until the savepoint is created and the job has been stopped + Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); + conf.set( + SavepointConfigOptions.SAVEPOINT_PATH, + savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); + } else { + jobClient.cancel(); + } + + // Wait until the job has been stopped + Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); + return conf; + } + + return null; + } + + /** + * Close the {@link JobClient} and wait for the job closure. + * + * @param jobClient the job to close + */ + public static void closeJobClient(JobClient jobClient) { + closeJobClient(jobClient, null); + } + + private static class MemoryLock implements TriggerLockFactory.Lock { + boolean locked = false; + + @Override + public boolean tryLock() { + if (locked) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java new file mode 100644 index 000000000000..051d09d92bad --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +class TestJdbcLockFactory extends TestLockFactoryBase { + @Override + TriggerLockFactory lockFactory() { + return lockFactory("tableName"); + } + + @Test + void testMultiTableLock() { + JdbcLockFactory other = lockFactory("tableName2"); + other.open((JdbcLockFactory) this.lockFactory); + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = other.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + private JdbcLockFactory lockFactory(String tableName) { + Map<String, String> properties = Maps.newHashMap(); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(INIT_LOCK_TABLES_PROPERTY, "true"); + + return new JdbcLockFactory( + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), + tableName, + properties); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java new file mode 100644 index 000000000000..bf9e86f2534d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +abstract class TestLockFactoryBase { + protected TriggerLockFactory lockFactory; + + abstract TriggerLockFactory lockFactory(); + + @BeforeEach + void before() { + this.lockFactory = lockFactory(); + lockFactory.open(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testTryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock1.tryLock()).isFalse(); + assertThat(lock2.tryLock()).isFalse(); + } + + @Test + void testUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + } + + @Test + void testNoConflictWithRecoveryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createRecoveryLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + @Test + void testDoubleUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + assertThat(lock.tryLock()).isFalse(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java index 876d64214560..3aee05322561 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -35,6 +34,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -161,7 +161,12 @@ void testSource() throws Exception { } // The first non-empty event should contain the expected value - return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + return newEvent.equals( + TableChange.builder() + .dataFileCount(1) + .dataFileSizeInBytes(size) + .commitCount(1) + .build()); }); } finally { closeJobClient(jobClient); @@ -297,17 +302,17 @@ void testMaxReadBack() { new MonitorSource.TableChangeIterator(tableLoader, null, 1); // For a single maxReadBack we only get a single change - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); // Expecting 2 commits/snapshots for maxReadBack=2 - assertThat(iterator.next().commitNum()).isEqualTo(2); + assertThat(iterator.next().commitCount()).isEqualTo(2); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // For maxReadBack Long.MAX_VALUE we get every change - assertThat(iterator.next().commitNum()).isEqualTo(3); + assertThat(iterator.next().commitCount()).isEqualTo(3); } @Test @@ -322,7 +327,7 @@ void testSkipReplace() { new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // Read the current snapshot - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot Table table = tableLoader.loadTable(); @@ -348,15 +353,19 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange List<DeleteFile> deleteFiles = Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); - long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); - - return new TableChange( - previous.dataFileNum() + dataFiles.size(), - previous.deleteFileNum() + deleteFiles.size(), - previous.dataFileSize() + dataSize, - previous.deleteFileSize() + deleteSize, - previous.commitNum() + 1); + long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); + + TableChange newChange = previous.copy(); + newChange.merge( + TableChange.builder() + .dataFileCount(dataFiles.size()) + .dataFileSizeInBytes(dataSize) + // Currently we only test with equality deletes + .eqDeleteFileCount(deleteFiles.size()) + .eqDeleteRecordCount(deleteRecordCount) + .commitCount(1) + .build()); + return newChange; } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java new file mode 100644 index 000000000000..fba4a12d9c6b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -0,0 +1,686 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +class TestTriggerManager extends OperatorTestBase { + private static final long DELAY = 10L; + private static final String NAME_1 = "name1"; + private static final String NAME_2 = "name2"; + private long processingTime = 0L; + private TriggerLockFactory lockFactory; + private TriggerLockFactory.Lock lock; + private TriggerLockFactory.Lock recoveringLock; + + @BeforeEach + void before() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + this.lockFactory = lockFactory(); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveringLock = lockFactory.createRecoveryLock(); + lock.unlock(); + recoveringLock.unlock(); + MetricsReporterFactoryForTests.reset(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testCommitCount() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 4); + } + } + + @Test + void testDataFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 0); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(5).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 4); + } + } + + @Test + void testDataFileSizeInBytes() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 3); + } + } + + @Test + void testPosDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 4); + } + } + + @Test + void testPosDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testEqDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 4); + } + } + + @Test + void testEqDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testTimeout() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + TableChange event = TableChange.builder().dataFileCount(1).commitCount(1).build(); + + // Wait for some time + testHarness.processElement(event, EVENT_TIME); + assertThat(testHarness.extractOutputValues()).isEmpty(); + + // Wait for the timeout to expire + long newTime = EVENT_TIME + Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // Send a new event + testHarness.setProcessingTime(newTime + 1); + testHarness.processElement(event, newTime); + + // No trigger yet + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Send a new event + newTime += Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + + // New trigger should arrive + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testStateRestore() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + testHarness.processElement( + TableChange.builder().dataFileCount(1).commitCount(1).build(), EVENT_TIME); + + assertThat(testHarness.extractOutputValues()).isEmpty(); + + state = testHarness.snapshot(1, EVENT_TIME); + } + + // Restore the state, write some more data, create a checkpoint, check the data which is written + manager = manager(tableLoader); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + // Arrives the first real change which triggers the recovery process + testHarness.processElement(TableChange.builder().commitCount(1).build(), EVENT_TIME_2); + assertTriggers( + testHarness.extractOutputValues(), + Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); + + // Remove the lock to allow the next trigger + recoveringLock.unlock(); + testHarness.setProcessingTime(EVENT_TIME_2); + // At this point the output contains the recovery trigger and the real trigger + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testMinFireDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, DELAY, 1); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // No new fire yet + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testLockCheckDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, 1, DELAY); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Create a lock to prevent execution, and check that there is no result + assertThat(lock.tryLock()).isTrue(); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // Remove the lock, and still no trigger + lock.unlock(); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + /** + * Simulating recovery scenarios where there is a leftover table lock, and ongoing maintenance + * task. + * + * @param locked if a lock exists on the table on job recovery + * @param runningTask is running and continues to run after job recovery + */ + @ParameterizedTest + @MethodSource("parametersForTestRecovery") + void testRecovery(boolean locked, boolean runningTask) throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + state = testHarness.snapshot(1, EVENT_TIME); + } + + if (locked) { + assertThat(lock.tryLock()).isTrue(); + } + + manager = manager(tableLoader); + List<Trigger> expected = Lists.newArrayListWithExpectedSize(3); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + ++processingTime; + expected.add(Trigger.recovery(processingTime)); + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Nothing happens until the recovery is finished + ++processingTime; + testHarness.setProcessingTime(processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + if (runningTask) { + // Simulate the action of the recovered maintenance task lock removal when it finishes + lock.unlock(); + } + + // Still no results as the recovery is ongoing + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it + // received recovery trigger + lock.unlock(); + recoveringLock.unlock(); + + // Emit only a single trigger + ++processingTime; + testHarness.setProcessingTime(processingTime); + // Releasing lock will create a new snapshot, and we receive this in the trigger + expected.add( + Trigger.create( + processingTime, + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), + 0)); + assertTriggers(testHarness.extractOutputValues(), expected); + } + } + + @Test + void testTriggerMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TableChange> source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink<Trigger> sink = new CollectingSink<>(); + + TriggerManager manager = + new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1, NAME_2), + Lists.newArrayList( + new TriggerEvaluator.Builder().commitCount(2).build(), + new TriggerEvaluator.Builder().commitCount(4).build()), + 1L, + 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // This one doesn't trigger - tests NOTHING_TO_TRIGGER + source.sendRecord(TableChange.builder().commitCount(1).build()); + + Awaitility.await() + .until( + () -> { + Long notingCounter = + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + return notingCounter != null && notingCounter.equals(1L); + }); + + // Trigger one of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(1).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(1L); + lock.unlock(); + + // Trigger both of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(2).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(2L); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + .isEqualTo(1L); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder<String, Long>() + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) + .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testRateLimiterMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TableChange> source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink<Trigger> sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1_000_000L, 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // The second trigger will be blocked + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + .equals(1L)); + + // Final check all the counters + assertCounters(1L, 0L); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testConcurrentRunMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TableChange> source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink<Trigger> sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1L, 1_000_000L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger - notice that we do not remove the lock after the trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // The second trigger will be blocked by the lock + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + .equals(1L)); + + // Final check all the counters + assertCounters(0L, 1L); + } finally { + closeJobClient(jobClient); + } + } + + private static Stream<Arguments> parametersForTestRecovery() { + return Stream.of( + Arguments.of(true, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false)); + } + + private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder<String, Long>() + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + rateLimiterTrigger) + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + concurrentRunTrigger) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .build()); + } + + private KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> harness( + TriggerManager manager) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new KeyedProcessOperator<>(manager), value -> true, Types.BOOLEAN); + } + + private void addEventAndCheckResult( + OneInputStreamOperatorTestHarness<TableChange, Trigger> testHarness, + TableChange event, + int expectedSize) + throws Exception { + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(event, processingTime); + assertThat(testHarness.extractOutputValues()).hasSize(expectedSize); + // Remove the lock to allow the next trigger + lock.unlock(); + } + + private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { + return new TriggerManager( + tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + } + + private TriggerManager manager( + TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { + return new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1), + Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), + minFireDelayMs, + lockCheckDelayMs); + } + + private TriggerManager manager(TableLoader tableLoader) { + return manager(tableLoader, new TriggerEvaluator.Builder().commitCount(2).build()); + } + + private static void assertTriggers(List<Trigger> expected, List<Trigger> actual) { + assertThat(actual).hasSize(expected.size()); + for (int i = 0; i < expected.size(); ++i) { + Trigger expectedTrigger = expected.get(i); + Trigger actualTrigger = actual.get(i); + assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); + assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); + assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); + if (expectedTrigger.table() == null) { + assertThat(actualTrigger.table()).isNull(); + } else { + Iterator<Snapshot> expectedSnapshots = expectedTrigger.table().snapshots().iterator(); + Iterator<Snapshot> actualSnapshots = actualTrigger.table().snapshots().iterator(); + while (expectedSnapshots.hasNext()) { + assertThat(actualSnapshots.hasNext()).isTrue(); + assertThat(expectedSnapshots.next().snapshotId()) + .isEqualTo(actualSnapshots.next().snapshotId()); + } + } + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List<StreamElement> transformsToStreamElement(Collection<Object> elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord<CommittableMessage<?>>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary<?> extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary<?>) value; + } + + static CommittableWithLineage<IcebergCommittable> extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage<IcebergCommittable>) value; + } +} 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/TestCompressionSettings.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 14f12422da96..3299e7a97776 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; @@ -33,55 +40,50 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; - private final Map<String, String> initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } + @Parameter(index = 0) + private Map<String, String> initProperties; - public TestCompressionSettings(Map<String, String> initProperties) { - this.initProperties = initProperties; + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; } - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); } - @Test + @TestTemplate public void testCompressionAvro() throws Exception { // No override provided Map<String, String> resultProperties = @@ -91,19 +93,17 @@ public void testCompressionAvro() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -119,11 +119,12 @@ public void testCompressionAvro() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionParquet() throws Exception { // No override provided Map<String, String> resultProperties = @@ -133,19 +134,19 @@ public void testCompressionParquet() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -161,11 +162,12 @@ public void testCompressionParquet() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionOrc() throws Exception { // No override provided Map<String, String> resultProperties = @@ -175,19 +177,18 @@ public void testCompressionOrc() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); } // Override compression to snappy and a different strategy @@ -203,12 +204,15 @@ public void testCompressionOrc() throws Exception { FlinkWriteOptions.COMPRESSION_STRATEGY.key(), "speed")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map<String, String> override) throws Exception { + private static OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map<String, String> override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -216,7 +220,7 @@ private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIce IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness<RowData, WriteResult> harness = + OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -227,7 +231,7 @@ private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIce private static Map<String, String> appenderProperties( Table table, TableSchema schema, Map<String, String> override) throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/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<Row> rows = createRows(""); - DataStream<Row> 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<String, String> 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<Row> leftRows = createRows("left-"); - DataStream<Row> 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<Row> rightRows = createRows("right-"); - DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List<Row> rows = createRows(""); - DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List<Row> rows = createRows(""); - DataStream<Row> 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<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream<RowData> 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> 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<Row> createBoundedSource(List<Row> rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List<Row> createRows(String prefix) { protected List<RowData> convertToRowData(List<Row> rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> 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..df8c3c79d3e3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,494 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +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<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> 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 testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + + @TestTemplate + public void testRangeDistributionWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List<Row> rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream<Row> dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + // sometimes + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource<Row> createRangeDistributionBoundedSource( + List<List<Row>> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List<List<Row>> createCharRows(int numOfCheckpoints, int countPerChar) { + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List<Row> rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List<List<Row>> createIntRows(int numOfCheckpoints, int maxId) { + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List<Row> rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map<String, String> 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<Row> leftRows = createRows("left-"); + DataStream<Row> 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<Row> rightRows = createRows("right-"); + DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> 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<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream<RowData> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java new file mode 100644 index 000000000000..a5f24e09a60b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test range distribution with bucketing partition column. Compared to hash distribution, range + * distribution is more general to handle bucketing column while achieving even distribution of + * traffic to writer tasks. + * + * <ul> + * <li><a href="https://github.com/apache/iceberg/pull/4228">keyBy on low cardinality</a> (e.g. + * 60) may not achieve balanced data distribution. + * <li>number of buckets (e.g. 60) is not divisible by the writer parallelism (e.g. 40). + * <li>number of buckets (e.g. 60) is smaller than the writer parallelism (e.g. 120). + * </ul> + */ +@Timeout(value = 30) +public class TestFlinkIcebergSinkRangeDistributionBucketing { + private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + // max supported parallelism is 16 (= 4 x 4) + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(4) + .setNumberSlotsPerTaskManager(4) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private static final int NUM_BUCKETS = 4; + private static final int NUM_OF_CHECKPOINTS = 4; + private static final int ROW_COUNT_PER_CHECKPOINT = 200; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "uuid", Types.UUIDType.get()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("uuid", NUM_BUCKETS).build(); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + private TableLoader tableLoader; + private Table table; + + @BeforeEach + public void before() throws IOException { + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name())); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + // Assuming ts is on ingestion/processing time. Writer only writes to 1 or 2 hours concurrently. + // Only sort on the bucket column to avoid each writer task writes to 60 buckets/files + // concurrently. + table.replaceSortOrder().asc(bucket("uuid", NUM_BUCKETS)).commit(); + } + + @AfterEach + public void after() throws Exception { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + /** number of buckets 4 matches writer parallelism of 4 */ + @Test + public void testBucketNumberEqualsToWriterParallelism() throws Exception { + testParallelism(4); + } + + /** number of buckets 4 is less than writer parallelism of 6 */ + @Test + public void testBucketNumberLessThanWriterParallelismNotDivisible() throws Exception { + testParallelism(6); + } + + /** number of buckets 4 is less than writer parallelism of 8 */ + @Test + public void testBucketNumberLessThanWriterParallelismDivisible() throws Exception { + testParallelism(8); + } + + /** number of buckets 4 is greater than writer parallelism of 3 */ + @Test + public void testBucketNumberHigherThanWriterParallelismNotDivisible() throws Exception { + testParallelism(3); + } + + /** number of buckets 4 is greater than writer parallelism of 2 */ + @Test + public void testBucketNumberHigherThanWriterParallelismDivisible() throws Exception { + testParallelism(2); + } + + private void testParallelism(int parallelism) throws Exception { + try (StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism)) { + + DataGeneratorSource<RowData> generatorSource = + new DataGeneratorSource<>( + new RowGenerator(), + ROW_COUNT_PER_CHECKPOINT * NUM_OF_CHECKPOINTS, + RateLimiterStrategy.perCheckpoint(ROW_COUNT_PER_CHECKPOINT), + FlinkCompatibilityUtil.toTypeInfo(ROW_TYPE)); + DataStream<RowData> dataStream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the oldest snapshot to the newest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Source rate limit per checkpoint cycle may not be super precise. + // There could be more checkpoint cycles and commits than planned. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(NUM_OF_CHECKPOINTS); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + assertThat(addedDataFiles) + .hasSizeLessThanOrEqualTo(maxAddedDataFilesPerCheckpoint(parallelism)); + } + } + } + + /** + * Traffic is not perfectly balanced across all buckets in the small sample size Range + * distribution of the bucket id may cross subtask boundary. Hence the number of committed data + * files per checkpoint maybe larger than writer parallelism or the number of buckets. But it + * should not be more than the sum of those two. Without range distribution, the number of data + * files per commit can be 4x of parallelism (as the number of buckets is 4). + */ + private int maxAddedDataFilesPerCheckpoint(int parallelism) { + return NUM_BUCKETS + parallelism; + } + + private static class RowGenerator implements GeneratorFunction<Long, RowData> { + // use constant timestamp so that all rows go to the same hourly partition + private final long ts = System.currentTimeMillis(); + + @Override + public RowData map(Long index) throws Exception { + // random uuid should result in relatively balanced distribution across buckets + UUID uuid = UUID.randomUUID(); + ByteBuffer uuidByteBuffer = ByteBuffer.allocate(16); + uuidByteBuffer.putLong(uuid.getMostSignificantBits()); + uuidByteBuffer.putLong(uuid.getLeastSignificantBits()); + return GenericRowData.of( + TimestampData.fromEpochMillis(ts), + uuidByteBuffer.array(), + StringData.fromString("row-" + index)); + } + } +} diff --git a/flink/v1.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..b283b8390a2b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -38,7 +39,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 +57,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 +89,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -185,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index fc33c2fea5e6..da4ac2ada677 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation<Row> ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation<Row> ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map<String, RowKind> ROW_KIND_MAP = + static final Map<String, RowKind> ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List<String> equalityFieldColumns, KeySelector<Row, Object> keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List<Snapshot> findValidSnapshots() { + List<Snapshot> findValidSnapshots() { List<Snapshot> validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List<Snapshot> findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable<Record> reader = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..a2e9be9303c3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -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; @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -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/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index ce1f208a4b07..c21c3d5cc21b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -45,28 +47,26 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkManifest { private static final Configuration CONF = new Configuration(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; private FileAppenderFactory<RowData> appenderFactory; private final AtomicInteger fileCount = new AtomicInteger(0); - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + assertThat(new File(tablePath).mkdir()).isTrue(); // Construct the iceberg table. table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); @@ -112,11 +112,11 @@ public void testIO() throws IOException { WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < 5; i++) { TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); } @@ -131,9 +131,11 @@ public void testUserProvidedManifestLocation() throws IOException { long checkpointId = 1; String flinkJobId = newFlinkJobId(); String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map<String, String> props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); @@ -144,21 +146,18 @@ public void testUserProvidedManifestLocation() throws IOException { () -> factory.create(checkpointId), table.spec()); - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } @@ -198,7 +197,7 @@ public void testVersionedSerializer() throws IOException { byte[] versionedSerializeData2 = SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); } @Test @@ -220,14 +219,13 @@ public void testCompatibility() throws IOException { DeltaManifests delta = SimpleVersionedSerialization.readVersionAndDeSerialize( DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); TestHelpers.assertEquals(manifest, delta.dataManifest()); List<DataFile> actualFiles = FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); + assertThat(actualFiles).hasSize(10); for (int i = 0; i < 10; i++) { TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..abdb2b5c79ae --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List<Object> parameters() { + List<Object> parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map<String, String> props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest<IcebergCommittable> commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest<IcebergCommittable> commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List<RowData> rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest<IcebergCommittable> commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map<String, String> summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary<IcebergCommittable> committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List<StreamElement> output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary<IcebergCommittable> committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List<StreamElement> output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary<IcebergCommittable> committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary<IcebergCommittable> committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List<StreamElement> output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary<IcebergCommittable> committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary<IcebergCommittable> committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List<StreamElement> output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary<IcebergCommittable> committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List<RowData> expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List<RowData> rows = Lists.newArrayList(); + List<RowData> tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List<RowData> tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List<RowData> rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List<RowData> expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List<Path> manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List<DataFile> dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List<StreamElement> output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List<StreamElement> output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List<StreamElement> output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List<StreamElement> output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary<IcebergCommittable> processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary<IcebergCommittable> committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage<IcebergCommittable> committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary<IcebergCommittable> processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory<RowData> createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List<Path> assertFlinkManifests(int expectedCount) throws IOException { + List<Path> manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List<RowData> rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory<RowData> appenderFactory, String filename, List<RowData> deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest<IcebergCommittable> buildCommitRequestFor( + String myJobID, long checkpoint, Collection<WriteResult> writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest<IcebergCommittable> commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer<CommittableMessage<IcebergCommittable>> { + + CommittableMessageSerializer<IcebergCommittable> serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<CommittableMessage<IcebergCommittable>> duplicate() { + return null; + } + + @Override + public CommittableMessage<IcebergCommittable> createInstance() { + return null; + } + + @Override + public CommittableMessage<IcebergCommittable> copy( + CommittableMessage<IcebergCommittable> from) { + return from; + } + + @Override + public CommittableMessage<IcebergCommittable> copy( + CommittableMessage<IcebergCommittable> from, CommittableMessage<IcebergCommittable> reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage<IcebergCommittable> record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage<IcebergCommittable> deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage<IcebergCommittable> deserialize( + CommittableMessage<IcebergCommittable> reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage<IcebergCommittable> deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot<CommittableMessage<IcebergCommittable>> snapshotConfiguration() { + return null; + } + }; +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 948c7b31430c..7808771d9887 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -139,8 +140,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -161,7 +161,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +177,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +194,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +206,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +235,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +246,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +289,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +300,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +341,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +354,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +367,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +381,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +406,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List<RowData> expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +418,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +454,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +481,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +504,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +518,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +536,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +550,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +575,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +588,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +612,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness<WriteResult, Void> harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +631,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +654,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness<WriteResult, Void> harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +681,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +707,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +720,14 @@ public void testBoundedStream() throws Exception { List<RowData> tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +740,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +751,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +791,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +801,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +833,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +865,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +880,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +894,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +911,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + * <ul> + * <li>A specific row is updated + * <li>The prepareSnapshotPreBarrier triggered + * <li>Checkpoint failed for reasons outside of the Iceberg connector + * <li>The specific row is updated again in the second checkpoint as well + * <li>Second snapshot is triggered, and finished + * </ul> + * + * <p>Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory<RowData> appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +996,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1008,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1027,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1045,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1062,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1079,8 +1178,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } @@ -1089,7 +1188,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness<WriteResult, Void> createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1208,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory<Void> - implements OneInputStreamOperatorFactory<WriteResult, Void> { + implements OneInputStreamOperatorFactory<FlinkWriteResult, Void> { private final String tablePath; private final String branch; private final PartitionSpec spec; 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/TestIcebergSink.java similarity index 64% 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/TestIcebergSink.java index 8cad35c859c4..7f355c1e8403 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/TestIcebergSink.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; @@ -26,12 +25,12 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -43,12 +42,11 @@ 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.sink.IcebergSink.Builder; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -56,18 +54,9 @@ 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); +public class TestIcebergSink extends TestFlinkIcebergSinkBase { private TableLoader tableLoader; @@ -80,7 +69,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @Parameter(index = 2) private boolean partitioned; - @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") public static Object[][] parameters() { return new Object[][] { {FileFormat.AVRO, 1, true}, @@ -99,7 +88,7 @@ public static Object[][] parameters() { } @BeforeEach - public void before() throws IOException { + void before() throws IOException { table = CATALOG_EXTENSION .catalog() @@ -113,7 +102,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); @@ -122,13 +111,13 @@ public void before() throws IOException { } @TestTemplate - public void testWriteRowData() throws Exception { + void testWriteRowData() throws Exception { List<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream<RowData> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - FlinkSink.forRowData(dataStream) + IcebergSink.forRowData(dataStream) .table(table) .tableLoader(tableLoader) .writeParallelism(parallelism) @@ -141,41 +130,18 @@ public void testWriteRowData() throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); } - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List<Row> rows = createRows(""); - DataStream<Row> 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 { + void testWriteRow() throws Exception { testWriteRow(null, DistributionMode.NONE); } @TestTemplate - public void testWriteRowWithTableSchema() throws Exception { + void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } @TestTemplate - public void testJobNoneDistributeMode() throws Exception { + void testJobNoneDistributeMode() throws Exception { table .updateProperties() .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) @@ -186,13 +152,13 @@ public void testJobNoneDistributeMode() throws Exception { if (parallelism > 1) { if (partitioned) { int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); } } } @TestTemplate - public void testJobHashDistributionMode() { + void testJobHashDistributionMode() { table .updateProperties() .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) @@ -204,7 +170,7 @@ public void testJobHashDistributionMode() { } @TestTemplate - public void testJobNullDistributionMode() throws Exception { + void testJobNullDistributionMode() throws Exception { table .updateProperties() .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) @@ -213,34 +179,52 @@ public void testJobNullDistributionMode() throws Exception { testWriteRow(null, null); if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); } } @TestTemplate - public void testPartitionWriteMode() throws Exception { + 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); + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); } } @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { + 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); + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); } } @TestTemplate - public void testTwoSinksInDisjointedDAG() throws Exception { + void testTwoSinksInDisjointedDAG() throws Exception { Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table leftTable = @@ -271,7 +255,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); @@ -282,12 +266,12 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env.fromCollection(leftRows, ROW_TYPE_INFO) .name("leftCustomSource") .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) .table(leftTable) .tableLoader(leftTableLoader) .tableSchema(SimpleDataUtil.FLINK_SCHEMA) .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") + .uidSuffix("leftIcebergSink") .append(); List<Row> rightRows = createRows("right-"); @@ -295,15 +279,15 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env.fromCollection(rightRows, ROW_TYPE_INFO) .name("rightCustomSource") .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + IcebergSink.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")) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) .append(); // Execute the program. @@ -313,55 +297,56 @@ public void testTwoSinksInDisjointedDAG() throws Exception { 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"); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); } @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { + void testOverrideWriteConfigWithUnknownDistributionMode() { Map<String, String> newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); List<Row> rows = createRows(""); DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } @TestTemplate - public void testOverrideWriteConfigWithUnknownFileFormat() { + void testOverrideWriteConfigWithUnknownFileFormat() { Map<String, String> newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); List<Row> rows = createRows(""); DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) .writeParallelism(parallelism) - .setAll(newProps); - + .setAll(newProps) + .uidSuffix("ingestion"); assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file format: UNRECOGNIZED"); } @TestTemplate - public void testWriteRowWithTableRefreshInterval() throws Exception { + void testWriteRowWithTableRefreshInterval() throws Exception { List<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream<RowData> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) @@ -370,7 +355,7 @@ public void testWriteRowWithTableRefreshInterval() throws Exception { Configuration flinkConf = new Configuration(); flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - FlinkSink.forRowData(dataStream) + IcebergSink.forRowData(dataStream) .table(table) .tableLoader(tableLoader) .flinkConf(flinkConf) @@ -383,4 +368,69 @@ public void testWriteRowWithTableRefreshInterval() throws Exception { // Assert the iceberg table's records. SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream<Row> dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List<List<Row>> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List<List<Record>> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream<Row> dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List<List<Row>> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List<List<Record>> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + new String( + deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + } + + protected void testChangeLogs( + List<String> equalityFieldColumns, + KeySelector<Row, Object> keySelector, + boolean insertAsUpsert, + List<List<Row>> elementsPerCheckpoint, + List<List<Record>> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream<Row> dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List<Snapshot> snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List<Record> expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java new file mode 100644 index 000000000000..93da40ab5c9a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index c05f08b82f51..e13721a9f170 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -22,11 +22,13 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -42,6 +44,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -58,52 +63,47 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; private Table table; - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table. Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,10 +111,11 @@ public void testWritingTable() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); checkpointId = checkpointId + 1; @@ -124,9 +125,10 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); @@ -145,49 +147,53 @@ public void testWritingTable() throws Exception { } } - @Test + @TestTemplate public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } } - @Test + @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); + assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); + assertThat(scanDataFiles()).hasSize(1); } private Set<String> scanDataFiles() throws IOException { @@ -211,9 +217,9 @@ private Set<String> scanDataFiles() throws IOException { } } - @Test + @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,45 +227,49 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); testHarness.endInput(); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testTableWithTargetFileSize() throws Exception { // Adjust the target-file-size in table properties. table @@ -276,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -284,13 +294,14 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); // Assert that the data file have the expected records. for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); + assertThat(dataFile.recordCount()).isEqualTo(1000); } // Commit the iceberg transaction. @@ -303,7 +314,7 @@ public void testTableWithTargetFileSize() throws Exception { SimpleDataUtil.assertTableRecords(table, records); } - @Test + @TestTemplate public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( @@ -329,7 +340,8 @@ public void testPromotedFlinkDataType() throws Exception { spec = PartitionSpec.unpartitioned(); } - String location = tempFolder.newFolder().getAbsolutePath(); + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); @@ -346,15 +358,16 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter() + private static List<WriteResult> getWriteResults(List<FlinkWriteResult> flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStr IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness<RowData, WriteResult> harness = + OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java index 79be96d20902..919fef579ab0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +37,7 @@ import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRowDataPartitionKey { private static final Schema SCHEMA = @@ -91,10 +92,10 @@ public void testNullPartitionValue() { for (RowData row : rows) { PartitionKey partitionKey = new PartitionKey(spec, schema); partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); + assertThat(partitionKey.size()).isEqualTo(1); String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); } } @@ -116,15 +117,15 @@ public void testPartitionWithOneNestedField() { PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); + assertThat(partitionKey1.size()).isEqualTo(1); - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); + assertThat(partitionKey2.size()).isEqualTo(1); - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); } } @@ -154,16 +155,16 @@ public void testPartitionMultipleNestedField() { Record record = (Record) records.get(i).get(0); pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); + assertThat(pk1.size()).isEqualTo(2); - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); + assertThat(pk2.size()).isEqualTo(2); - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); } } @@ -190,19 +191,18 @@ public void testPartitionValueTypes() { pk.partition(rowWrapper.wrap(row)); expectedPK.partition(recordWrapper.wrap(record)); - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } @@ -232,19 +232,18 @@ public void testNestedPartitionValues() { pk.partition(rowWrapper.wrap(rows.get(j))); expectedPK.partition(recordWrapper.wrap(records.get(j))); - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 112dbb511310..8bfd6cb3d043 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; @@ -30,6 +33,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -39,69 +45,62 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestTaskWriters { private static final Configuration CONF = new Configuration(); private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; - private Table table; + @Parameter(index = 1) + private boolean partitioned; - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } + private Table table; - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table with the specified file format. Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); // Close again. taskWriter.close(); dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); } } - @Test + @TestTemplate public void testCloseTwice() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -111,16 +110,16 @@ public void testCloseTwice() throws IOException { int expectedFiles = partitioned ? 2 : 1; DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } } } - @Test + @TestTemplate public void testAbort() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -130,16 +129,16 @@ public void testAbort() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); } } } - @Test + @TestTemplate public void testCompleteFiles() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "a")); @@ -149,14 +148,14 @@ public void testCompleteFiles() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); @@ -176,7 +175,7 @@ public void testCompleteFiles() throws IOException { } } - @Test + @TestTemplate public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(4)) { List<RowData> rows = Lists.newArrayListWithCapacity(8000); @@ -193,7 +192,7 @@ public void testRollingWithTargetFileSize() throws IOException { } DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); + assertThat(dataFiles).hasSize(8); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { @@ -206,7 +205,7 @@ public void testRollingWithTargetFileSize() throws IOException { } } - @Test + @TestTemplate public void testRandomData() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable<RowData> rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java index c7e9f19abb02..5910bd685510 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -53,8 +53,10 @@ private Fixtures() {} new SortKeySerializer(SCHEMA, SORT_ORDER); public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = new DataStatisticsSerializer(SORT_KEY_SERIALIZER); - public static final AggregatedStatisticsSerializer AGGREGATED_STATISTICS_SERIALIZER = - new AggregatedStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); public static final Map<String, SortKey> CHAR_KEYS = createCharKeys(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 52485a3c67d9..8322ce683768 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -24,7 +24,7 @@ import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -36,7 +36,7 @@ public void receiveNewerStatisticsEvent(StatisticsType type) { StatisticsEvent checkpoint1Subtask0StatisticsEvent = createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -86,7 +86,8 @@ public void receiveNewerStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); } // checkpoint 2 remains @@ -115,7 +116,7 @@ public void receiveOlderStatisticsEventTest(StatisticsType type) { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); @@ -189,7 +190,14 @@ public void receiveOlderStatisticsEventTest(StatisticsType type) { CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 4L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); } } @@ -207,7 +215,7 @@ public void receiveCompletedStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -246,7 +254,14 @@ public void receiveCompletedStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("a"), 3L, CHAR_KEYS.get("b"), 3L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); } StatisticsEvent checkpoint2Subtask0DataStatisticEvent = @@ -280,7 +295,8 @@ public void receiveCompletedStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); } } @@ -307,7 +323,7 @@ public void coordinatorSwitchToSketchOverThreshold() { 1L, CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -349,8 +365,14 @@ public void coordinatorSwitchToSketchOverThreshold() { assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); - assertThat(completedStatistics.rangeBounds()) - .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); } @Test @@ -376,7 +398,7 @@ public void coordinatorMapOperatorSketch() { 1L, CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -419,8 +441,14 @@ public void coordinatorMapOperatorSketch() { assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); - assertThat(completedStatistics.rangeBounds()) - .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); } private AggregatedStatisticsTracker createTracker(StatisticsType type) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java new file mode 100644 index 000000000000..4ee9888934a8 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestCompletedStatisticsSerializer extends SerializerTestBase<CompletedStatistics> { + + @Override + protected TypeSerializer<CompletedStatistics> createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<CompletedStatistics> getTypeClass() { + return CompletedStatistics.class; + } + + @Override + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( + 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index fe1d07c3286a..a08a8a73e80c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -19,17 +19,24 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.time.Duration; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -45,7 +52,7 @@ public void before() throws Exception { } private void tasksReady(DataStatisticsCoordinator coordinator) { - setAllTasksReady(Fixtures.NUM_SUBTASKS, coordinator, receivingTasks); + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); } @ParameterizedTest @@ -106,20 +113,95 @@ public void testDataStatisticsEventHandling(StatisticsType type) throws Exceptio waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify global data statistics is the aggregation of all subtasks data statistics - AggregatedStatistics aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.checkpointId()).isEqualTo(1L); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + Map<SortKey, Long> keyFrequency = + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 3L, + CHAR_KEYS.get("c"), 5L); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(NUM_SUBTASKS, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), 2L, - CHAR_KEYS.get("b"), 3L, - CHAR_KEYS.get("c"), 5L)); + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.checkpointId()).isEqualTo(1L); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } + } + + @Test + public void testRequestGlobalStatisticsEventHandling() throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); } } @@ -154,10 +236,11 @@ static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordin private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { return new DataStatisticsCoordinator( OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, Fixtures.NUM_SUBTASKS), + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), Fixtures.SCHEMA, Fixtures.SORT_ORDER, - Fixtures.NUM_SUBTASKS, - type); + NUM_SUBTASKS, + type, + 0.0d); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index 966c8474b42b..6317f2bfde18 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -19,16 +19,19 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; @@ -47,7 +50,7 @@ public void before() { @ParameterizedTest @EnumSource(StatisticsType.class) public void testCheckpointAndReset(StatisticsType type) throws Exception { - DataStatisticsCoordinatorProvider provider = createProvider(type); + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) provider.create( @@ -62,42 +65,39 @@ public void testCheckpointAndReset(StatisticsType type) throws Exception { // Handle events from operators for checkpoint 1 StatisticsEvent checkpoint1Subtask0StatisticsEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c")); + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent( - type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); // Verify checkpoint 1 global data statistics - assertThat(dataStatisticsCoordinator.completedStatistics()).isNotNull(); - AggregatedStatistics aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + Map<SortKey, Long> checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), - 1L, - CHAR_KEYS.get("b"), - 1L, - CHAR_KEYS.get("c"), - 1L, - CHAR_KEYS.get("d"), - 1L, - CHAR_KEYS.get("e"), - 1L)); + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); } byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); @@ -114,42 +114,54 @@ public void testCheckpointAndReset(StatisticsType type) throws Exception { TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); // Verify checkpoint 2 global data statistics - aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + Map<SortKey, Long> checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L)); + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); } + waitForCheckpoint(2L, dataStatisticsCoordinator); // Reset coordinator to checkpoint 1 coordinator.resetToCheckpoint(1L, checkpoint1Bytes); DataStatisticsCoordinator restoredDataStatisticsCoordinator = (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - aggregatedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); // Verify restored data statistics if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), - 1L, - CHAR_KEYS.get("b"), - 1L, - CHAR_KEYS.get("c"), - 1L, - CHAR_KEYS.get("d"), - 1L, - CHAR_KEYS.get("e"), - 1L)); + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); } } } @@ -161,13 +173,15 @@ private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator co return future.get(); } - private static DataStatisticsCoordinatorProvider createProvider(StatisticsType type) { + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { return new DataStatisticsCoordinatorProvider( "DataStatisticsCoordinatorProvider", OPERATOR_ID, Fixtures.SCHEMA, Fixtures.SORT_ORDER, - Fixtures.NUM_SUBTASKS, - type); + downstreamParallelism, + type, + 0.0); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 6c864791a1bc..bc248b778184 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -19,12 +19,16 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; @@ -56,7 +60,10 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; public class TestDataStatisticsOperator { @@ -75,15 +82,22 @@ public void before() throws Exception { new TestTaskStateManager()); } - private DataStatisticsOperator createOperator(StatisticsType type) throws Exception { + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + return createOperator(type, downstreamParallelism, mockGateway); + } + + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { DataStatisticsOperator operator = new DataStatisticsOperator( "testOperator", Fixtures.SCHEMA, Fixtures.SORT_ORDER, mockGateway, - Fixtures.NUM_SUBTASKS, + downstreamParallelism, type); operator.setup( new OneInputStreamTask<String, String>(env), @@ -96,7 +110,7 @@ private DataStatisticsOperator createOperator(StatisticsType type) throws Except @ParameterizedTest @EnumSource(StatisticsType.class) public void testProcessElement(StatisticsType type) throws Exception { - DataStatisticsOperator operator = createOperator(type); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); @@ -125,7 +139,7 @@ public void testProcessElement(StatisticsType type) throws Exception { @ParameterizedTest @EnumSource(StatisticsType.class) public void testOperatorOutput(StatisticsType type) throws Exception { - DataStatisticsOperator operator = createOperator(type); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = createHarness(operator)) { testHarness.processElement( @@ -149,35 +163,47 @@ public void testOperatorOutput(StatisticsType type) throws Exception { } } + private static Stream<Arguments> provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testRestoreState(StatisticsType type) throws Exception { + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { Map<SortKey, Long> keyFrequency = ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; - DataStatisticsOperator operator = createOperator(type); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness1 = createHarness(operator)) { - AggregatedStatistics statistics; + GlobalStatistics statistics; if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - statistics = AggregatedStatistics.fromKeyFrequency(1L, keyFrequency); + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); } else { - statistics = AggregatedStatistics.fromRangeBounds(1L, rangeBounds); + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); } StatisticsEvent event = - StatisticsEvent.createAggregatedStatisticsEvent( - 1L, statistics, Fixtures.AGGREGATED_STATISTICS_SERIALIZER); + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); operator.handleOperatorEvent(event); - AggregatedStatistics globalStatistics = operator.globalStatistics(); + GlobalStatistics globalStatistics = operator.globalStatistics(); assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.keyFrequency()).isEqualTo(keyFrequency); + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); assertThat(globalStatistics.rangeBounds()).isNull(); } else { - assertThat(globalStatistics.keyFrequency()).isNull(); + assertThat(globalStatistics.mapAssignment()).isNull(); assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); } @@ -186,19 +212,28 @@ public void testRestoreState(StatisticsType type) throws Exception { // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator restoredOperator = createOperator(type); + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); - AggregatedStatistics globalStatistics = restoredOperator.globalStatistics(); + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.keyFrequency()).isEqualTo(keyFrequency); + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); assertThat(globalStatistics.rangeBounds()).isNull(); } else { - assertThat(globalStatistics.keyFrequency()).isNull(); + assertThat(globalStatistics.mapAssignment()).isNull(); assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); } } @@ -207,7 +242,7 @@ public void testRestoreState(StatisticsType type) throws Exception { @SuppressWarnings("unchecked") @Test public void testMigrationWithLocalStatsOverThreshold() throws Exception { - DataStatisticsOperator operator = createOperator(StatisticsType.Auto); + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); @@ -259,7 +294,7 @@ public void testMigrationWithLocalStatsOverThreshold() throws Exception { @SuppressWarnings("unchecked") @Test public void testMigrationWithGlobalSketchStatistics() throws Exception { - DataStatisticsOperator operator = createOperator(StatisticsType.Auto); + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); @@ -272,12 +307,12 @@ public void testMigrationWithGlobalSketchStatistics() throws Exception { .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); // received global statistics with sketch type - AggregatedStatistics globalStatistics = - AggregatedStatistics.fromRangeBounds( + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); operator.handleOperatorEvent( - StatisticsEvent.createAggregatedStatisticsEvent( - 1L, globalStatistics, Fixtures.AGGREGATED_STATISTICS_SERIALIZER)); + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); int reservoirSize = SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); @@ -310,7 +345,7 @@ private OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> createHar dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); harness.setup( new StatisticsOrRecordSerializer( - Fixtures.AGGREGATED_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); harness.open(); return harness; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase<GlobalStatistics> { + + @Override + protected TypeSerializer<GlobalStatistics> createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<GlobalStatistics> getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d47e41768072..d5a0bebc74e7 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -89,39 +90,35 @@ private static SortKey[] initSortKeys() { @Test public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 0.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // each task should get targeted weight of 100 (=800/8) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -143,19 +140,20 @@ public void testEvenlyDividableNoClosingFileCost() { Pair.of(100L, 1), 7, Pair.of(100L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 5.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 100 before close file cost factored in. // close file cost is 5 = 5% * 100. @@ -164,35 +162,30 @@ public void testEvenlyDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight with close cost per subtask is 110 (880/8) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -215,51 +208,48 @@ public void testEvenlyDividableWithClosingFileCost() { Pair.of(100L, 2), 7, Pair.of(75L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 0.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 // each task should get targeted weight of 89 = ceiling(800/9) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -283,19 +273,20 @@ public void testNonDividableNoClosingFileCost() { Pair.of(89L, 2), 8, Pair.of(88L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 5.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 89 before close file cost factored in. // close file cost is 5 (= 5% * 89) per file. @@ -304,35 +295,31 @@ public void testNonDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight per subtask is 98 ceiling(880/9) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -357,40 +344,39 @@ public void testNonDividableWithClosingFileCost() { Pair.of(88L, 2), 8, Pair.of(61L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); // drift threshold is high for non-dividable scenario with close cost validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); } private static Map<Integer, Pair<AtomicLong, Set<RowData>>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { + MapRangePartitioner partitioner, int numPartitions, Map<SortKey, Long> mapStatistics) { // The Map key is the subtaskId. // For the map value pair, the first element is the count of assigned and // the second element of Set<String> is for the set of assigned keys. Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair<AtomicLong, Set<RowData>> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair<AtomicLong, Set<RowData>> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); return partitionResults; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/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<Integer> 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<Integer> 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 + * + * <ul> + * <li>age <= 15 + * <li>age > 15 && age <= 32 + * <li>age >32 && age <= 60 + * <li>age > 60 + * </ul> + */ + 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 435748281f83..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 @@ -53,9 +56,9 @@ public void testOperatorReservoirSize() { @Test public void testRangeBoundsOneChannel() { assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -70,9 +73,9 @@ public void testRangeBoundsOneChannel() { @Test public void testRangeBoundsDivisible() { assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -88,9 +91,9 @@ public void testRangeBoundsDivisible() { public void testRangeBoundsNonDivisible() { // step is 3 = ceiling(11/4) assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -111,9 +114,9 @@ public void testRangeBoundsNonDivisible() { public void testRangeBoundsSkipDuplicates() { // step is 3 = ceiling(11/4) assertThat( - SketchUtil.determineBounds( + 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/TestSortKeySerializerSnapshot.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..c0f688f2589e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER); + TypeSerializerSnapshot<SortKey> snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer<SortKey> restoredSerializer = snapshot.restoreSerializer(); + + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + + @Test + public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot<SortKey> original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/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/TableSourceTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java new file mode 100644 index 000000000000..f89d63ac73e3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/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/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index bc7194e38088..14131d9e96d5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -24,12 +24,12 @@ import java.util.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestFlinkSourceConfig extends TestFlinkTableSource { +public class TestFlinkSourceConfig extends TableSourceTestBase { private static final String TABLE = "test_table"; - @Test + @TestTemplate public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) @@ -37,7 +37,7 @@ public void testFlinkSessionConfig() { .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } - @Test + @TestTemplate public void testFlinkHintConfig() { List<Row> result = sql( @@ -46,10 +46,13 @@ public void testFlinkHintConfig() { assertThat(result).hasSize(3); } - @Test + @TestTemplate public void testReadOptionHierarchy() { getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List<Row> result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index facbdcaaa533..18528c789114 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -21,78 +21,16 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; -import java.io.IOException; import java.util.List; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestFlinkTableSource extends TestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - dropDatabase(DATABASE_NAME, true); - dropCatalog(CATALOG_NAME, true); - } - - @Test +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate public void testLimitPushDown() { assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) @@ -121,7 +59,7 @@ public void testLimitPushDown() { assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } - @Test + @TestTemplate public void testNoFilterPushDown() { String sql = String.format("SELECT * FROM %s ", TABLE_NAME); List<Row> result = sql(sql); @@ -133,7 +71,7 @@ public void testNoFilterPushDown() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDownEqual() { String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -147,7 +85,7 @@ public void testFilterPushDownEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); @@ -156,7 +94,7 @@ public void testFilterPushDownEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownEqualLiteralOnLeft() { String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -170,7 +108,7 @@ public void testFilterPushDownEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqual() { String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") != 1"; @@ -187,7 +125,7 @@ public void testFilterPushDownNoEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); @@ -196,7 +134,7 @@ public void testFilterPushDownNoEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownAnd() { String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); @@ -211,7 +149,7 @@ public void testFilterPushDownAnd() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownOr() { String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; @@ -229,7 +167,7 @@ public void testFilterPushDownOr() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThan() { String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 1"; @@ -247,7 +185,7 @@ public void testFilterPushDownGreaterThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); @@ -256,7 +194,7 @@ public void testFilterPushDownGreaterThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanLiteralOnLeft() { String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 3"; @@ -274,7 +212,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqual() { String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 2"; @@ -292,7 +230,7 @@ public void testFilterPushDownGreaterThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); @@ -301,7 +239,7 @@ public void testFilterPushDownGreaterThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 2"; @@ -319,7 +257,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThan() { String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 2"; @@ -334,7 +272,7 @@ public void testFilterPushDownLessThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); @@ -343,7 +281,7 @@ public void testFilterPushDownLessThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanLiteralOnLeft() { String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 2"; @@ -358,7 +296,7 @@ public void testFilterPushDownLessThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqual() { String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 1"; @@ -373,7 +311,7 @@ public void testFilterPushDownLessThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); @@ -382,7 +320,7 @@ public void testFilterPushDownLessThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualLiteralOnLeft() { String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 3"; @@ -397,7 +335,7 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; @@ -413,7 +351,7 @@ public void testFilterPushDownIn() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownInNull() { String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); @@ -430,7 +368,7 @@ public void testFilterPushDownInNull() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); @@ -444,7 +382,7 @@ public void testFilterPushDownNotIn() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List<Row> resultGT = sql(sqlNotInNull); @@ -455,7 +393,7 @@ public void testFilterPushDownNotInNull() { .isNull(); } - @Test + @TestTemplate public void testFilterPushDownIsNotNull() { String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); String expectedFilter = "not_null(ref(name=\"data\"))"; @@ -473,7 +411,7 @@ public void testFilterPushDownIsNotNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIsNull() { String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); String expectedFilter = "is_null(ref(name=\"data\"))"; @@ -488,7 +426,7 @@ public void testFilterPushDownIsNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); @@ -503,7 +441,7 @@ public void testFilterPushDownNot() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); @@ -522,7 +460,7 @@ public void testFilterPushDownBetween() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownNotBetween() { String sqlNotBetween = String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); @@ -538,7 +476,7 @@ public void testFilterPushDownNotBetween() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLike() { String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; @@ -565,7 +503,7 @@ public void testFilterPushDownLike() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; @@ -604,7 +542,7 @@ public void testFilterNotPushDownLike() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDown2Literal() { String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); List<Row> result = sql(sql2Literal); @@ -616,7 +554,7 @@ public void testFilterPushDown2Literal() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testSqlParseNaN() { // todo add some test case to test NaN } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -130,11 +128,8 @@ protected List<Row> run( sourceBuilder.properties(options); DataStream<Row> stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 0337f3597053..4e649d15b1ce 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -18,10 +18,10 @@ */ package org.apache.iceberg.flink.source; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Locale; import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -30,10 +30,12 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.AvroSchemaUtil; @@ -43,68 +45,64 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedGenericRecord { + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } - private final FileFormat fileFormat; - private final int parallelism; + @Parameter(index = 0) + private FileFormat fileFormat; - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } + @Parameter(index = 1) + private int parallelism; - @Test + @Parameter(index = 2) + private boolean useConverter; + + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { String dateStr = "2020-03-20"; Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); @@ -112,19 +110,19 @@ public void testPartitionedTable() throws Exception { expectedRecords.get(i).setField("dt", dateStr); } - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testProjection() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); // select the "data" field (fieldId == 1) Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); @@ -149,29 +147,20 @@ private List<Row> run( Configuration config = new Configuration(); config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { + try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { tableLoader.open(); table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder<GenericRecord> sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder<GenericRecord> sourceBuilder = - IcebergSource.<GenericRecord>builder() - .tableLoader(catalogResource.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -179,7 +168,6 @@ private List<Row> run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -200,4 +188,35 @@ private List<Row> run( return Lists.newArrayList(iter); } } + + private IcebergSource.Builder<GenericRecord> createSourceBuilderWithReaderFunction( + Table table, Schema projected, List<Expression> filters, Configuration config) { + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + projected, + null, + false, + table.io(), + table.encryption(), + filters); + + return IcebergSource.<GenericRecord>builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder<GenericRecord> createSourceBuilderWithConverter( + Table table, Schema readSchema, Configuration config) { + AvroGenericRecordConverter converter = + AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); + return IcebergSource.forOutputType(converter) + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 5765b73a1f63..749cbf89338a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -38,7 +39,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; @@ -46,45 +48,43 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceContinuous { public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(METRIC_REPORTER); + @TempDir protected Path temporaryFolder; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private final AtomicLong randomSeed = new AtomicLong(0L); @Test public void testTableScanThenIncremental() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -97,27 +97,27 @@ public void testTableScanThenIncremental() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -126,22 +126,24 @@ public void testTableScanThenIncremental() throws Exception { @Test public void testTableScanThenIncrementalAfterExpiration() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot2 List<Record> batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - Assert.assertEquals(1, tableResource.table().history().size()); + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); ScanContext scanContext = ScanContext.builder() @@ -150,8 +152,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { @@ -159,17 +161,17 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List<Record> initialRecords = Lists.newArrayList(); initialRecords.addAll(batch1); initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -178,16 +180,18 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { @Test public void testEarliestSnapshot() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -202,43 +206,46 @@ public void testEarliestSnapshot() throws Exception { List<Row> result1 = waitForResult(iter, 4); List<Record> combinedBatch0AndBatch1 = Lists.newArrayList(batch0); combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } } @Test - public void testLatestSnapshot() throws Exception { + public void testLatestSnapshot(@InjectClusterClient ClusterClient<?> clusterClient) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -252,29 +259,29 @@ public void testLatestSnapshot() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // we want to make sure job is running first so that enumerator can // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + waitUntilJobIsRunning(clusterClient); // inclusive behavior for starting snapshot List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -283,19 +290,21 @@ public void testLatestSnapshot() throws Exception { @Test public void testSpecificSnapshotId() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); ScanContext scanContext = ScanContext.builder() @@ -308,25 +317,25 @@ public void testSpecificSnapshotId() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -335,22 +344,24 @@ public void testSpecificSnapshotId() throws Exception { @Test public void testSpecificSnapshotTimestamp() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); // sleep for 2 ms to make sure snapshot1 has a higher timestamp value Thread.sleep(2); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); ScanContext scanContext = ScanContext.builder() @@ -364,25 +375,25 @@ public void testSpecificSnapshotTimestamp() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // consume data from snapshot1 List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -392,27 +403,30 @@ public void testSpecificSnapshotTimestamp() throws Exception { public void testReadingFromBranch() throws Exception { String branch = "b1"; GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); List<Record> batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchBase); // create branch - tableResource + TABLE_EXTENSION .table() .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) .commit(); // snapshot1 to branch List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch1); // snapshot2 to branch List<Record> batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch2); List<Record> branchExpectedRecords = Lists.newArrayList(); @@ -432,25 +446,26 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); // snapshot3 to branch List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); // snapshot4 to branch List<Record> batch4 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch4); List<Row> result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); } // read only from main branch. Should contain only the first snapshot @@ -463,14 +478,14 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); List<Record> batchMain2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchMain2); resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); } } @@ -479,7 +494,7 @@ public void testValidation() { assertThatThrownBy( () -> IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(true) .endTag("tag") @@ -495,7 +510,7 @@ private DataStream<Row> createStream(ScanContext scanContext) throws Exception { DataStream<Row> stream = env.fromSource( IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(scanContext.isStreaming()) .streamingStartingStrategy(scanContext.streamingStartingStrategy()) @@ -507,7 +522,7 @@ private DataStream<Row> createStream(ScanContext scanContext) throws Exception { WatermarkStrategy.noWatermarks(), "icebergSource", TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); return stream; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index a378a0b93be3..938ae4d9bb0a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Path; import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,6 +31,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; @@ -41,7 +43,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -51,53 +56,67 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; - +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 120) public class TestIcebergSourceFailover { // Parallelism higher than 1, but lower than the number of splits used by some of our tests // The goal is to allow some splits to remain in the enumerator when restoring the state private static final int PARALLELISM = 2; private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } protected IcebergSource.Builder<RowData> sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) // Prevent combining splits .set( @@ -119,13 +138,13 @@ protected void assertRecords(Table table, List<Record> expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Disabled("Disabled for now as it is flaky on CI") @Test - public void testBoundedWithSavepoint() throws Exception { + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient<?> clusterClient) + throws Exception { List<Record> expectedRecords = Lists.newArrayList(); - Table sinkTable = sinkTableResource.table(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List<Record> records = generateRecords(2, i); expectedRecords.addAll(records); @@ -141,13 +160,8 @@ public void testBoundedWithSavepoint() throws Exception { // Write something, but do not finish before checkpoint is created RecordCounterToWait.waitForCondition(); CompletableFuture<String> savepoint = - miniClusterResource - .getClusterClient() - .stopWithSavepoint( - jobId, - false, - TEMPORARY_FOLDER.newFolder().toPath().toString(), - SavepointFormatType.CANONICAL); + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint @@ -170,19 +184,21 @@ public void testBoundedWithSavepoint() throws Exception { @Test public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); } - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { List<Record> expectedRecords = Lists.newArrayList(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List<Record> records = generateRecords(2, i); expectedRecords.addAll(records); @@ -197,29 +213,27 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio JobID jobId = jobClient.getJobID(); RecordCounterToWait.waitForCondition(); - triggerFailover( - failoverType, - jobId, - RecordCounterToWait::continueProcessing, - miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } @Test public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); } - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); List<Record> expectedRecords = Lists.newArrayList(); List<Record> batch = generateRecords(2, 0); @@ -247,8 +261,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); @@ -260,13 +274,13 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep expectedRecords.addAll(records); dataAppender.appendToTable(records); if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, () -> {}, miniCluster); } } // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { @@ -286,8 +300,8 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); } @@ -295,6 +309,20 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ + private static void runTestWithNewMiniCluster(ThrowingConsumer<MiniCluster, Exception> testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + private enum FailoverType { NONE, TM, diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index cde39ec98d61..4f61d2f7308a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -49,11 +50,24 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + @Override protected IcebergSource.Builder<RowData> sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) // Prevent combining splits diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream<Row> dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector<Row> collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator<Row> iterator = collector.getOutput()) { + List<Row> result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 75f0a785a8c5..548940a842ce 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -53,7 +53,12 @@ public class TestIcebergSourceSql extends TestSqlBase { public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 8341c8074f21..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,9 +18,10 @@ */ 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; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -37,6 +38,7 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.InMemoryReporter; @@ -49,24 +51,24 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; @@ -74,25 +76,24 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int RECORD_NUM_FOR_2_SPLITS = 200; private static final ConcurrentMap<Long, Integer> WINDOWS = Maps.newConcurrentMap(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) .withHaLeadershipControl() .build()); - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); /** * This is an integration test for watermark handling and windowing. Integration testing the @@ -259,7 +260,7 @@ public void apply( * the readers continue reading. */ @Test - public void testThrottling() throws Exception { + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { GenericAppenderHelper dataAppender = appender(); // Generate records in advance @@ -310,8 +311,7 @@ public void testThrottling() throws Exception { try (CloseableIterator<RowData> resultIterator = stream.collectAsync()) { JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); // Insert the first data into the table dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); @@ -353,7 +353,7 @@ public void testThrottling() throws Exception { protected IcebergSource<RowData> source() { return IcebergSource.<RowData>builder() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) .splitSize(100L) @@ -377,7 +377,7 @@ protected Record generateRecord(int minutes, String str) { private Optional<Gauge<Long>> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() + return REPORTER.findMetrics(jobID, metricsName).values().stream() .map(m -> (Gauge<Long>) m) .filter(m -> m.getValue() == withValue) .findFirst(); @@ -389,7 +389,7 @@ private GenericAppenderHelper appender() { hadoopConf.set("write.parquet.page-size-bytes", "64"); hadoopConf.set("write.parquet.row-group-size-bytes", "64"); return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); } private static RowData row(long time, long count) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..05a08c24d8d0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.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.TaskInfo; +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.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static 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<Row> 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<Row> 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<Row, Row> { + @Override + public Row map(Row row) throws Exception { + // Simulate slow subtask 0 with attempt 0 + TaskInfo taskInfo = getRuntimeContext().getTaskInfo(); + if (taskInfo.getIndexOfThisSubtask() == 0 && taskInfo.getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), + getRuntimeContext().getTaskInfo().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java index 25ecec23d216..ce9054ad49b6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowDelta; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; @@ -40,20 +47,19 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestProjectMetaColumn { - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Iterable<Object[]> parameters() { return Lists.newArrayList( new Object[] {FileFormat.PARQUET}, @@ -61,13 +67,9 @@ public static Iterable<Object[]> parameters() { new Object[] {FileFormat.AVRO}); } - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, @@ -89,7 +91,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); + assertThat(rowData).isInstanceOf(GenericRowData.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); @@ -97,20 +99,20 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); } - @Test + @TestTemplate public void testV1SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(1); } - @Test + @TestTemplate public void testV2SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(2); } - @Test + @TestTemplate public void testV2RemoveMetaColumn() throws Exception { // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); @@ -132,7 +134,7 @@ public void testV2RemoveMetaColumn() throws Exception { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); + assertThat(rowData).isInstanceOf(RowDataProjection.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java index 317301260f66..b701419a7499 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSourceUtil { @Test @@ -30,31 +31,31 @@ public void testInferedParallelism() throws IOException { Configuration configuration = new Configuration(); // Empty table, infer parallelism should be at least 1 int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits // num : 2 parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + assertThat(parallelism).isEqualTo(2); // 2 splits and limit is 1 , max infer parallelism is default 100, // which is greater than splits num and limit, the parallelism is the limit value : 1 parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : // 1 parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); } } 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.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List<List<Record>> 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<RowData> dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List<RowData> result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 000000000000..952255a52b7c --- /dev/null +++ b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests 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 2509371bdb8b..294c88d90709 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 @@ -66,13 +66,15 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } - testImplementation libs.flink117.connector.test.utils - testImplementation libs.flink117.core - testImplementation libs.flink117.runtime - testImplementation(libs.flink117.test.utilsjunit) { + implementation libs.datasketches + + 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' } @@ -117,7 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core - testImplementation libs.junit.vintage.engine + testImplementation libs.sqlite.jdbc } test { @@ -167,7 +169,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 @@ -177,13 +179,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 94% 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 3b2c74fd6ece..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 @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.nio.charset.StandardCharsets; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -28,6 +29,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -67,6 +70,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator<StructLike> SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -83,10 +88,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List<Integer> keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; @@ -134,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<String, String> properties = ImmutableMap.builder(); String location = null; for (Map.Entry<String, String> 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 91% 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 index 7167859e600c..d5eea6706b39 100644 --- 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 @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public <E extends Enum<E>> EnumConfParser<E> enumConfParser(Class<E> enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser<DoubleConfParser, Double> { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser<StringConfParser, String> { private String defaultValue; 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 89% 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 index ca7b1120bc81..a31902d49a8b 100644 --- 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 @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.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 82% 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 index df73f2e09cac..c35286774874 100644 --- 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 @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption<String> DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption<String> RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption<Double> RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption<String> BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); 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/JdbcLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java new file mode 100644 index 000000000000..f22be33aeae6 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java @@ -0,0 +1,319 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JdbcLockFactory implements TriggerLockFactory { + private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); + + @VisibleForTesting + static final String INIT_LOCK_TABLES_PROPERTY = "flink-maintenance.lock.jdbc.init-lock-tables"; + + private static final String LOCK_TABLE_NAME = "flink_maintenance_lock"; + private static final int LOCK_ID_MAX_LENGTH = 100; + private static final String CREATE_LOCK_TABLE_SQL = + String.format( + "CREATE TABLE %s " + + "(LOCK_TYPE CHAR(1) NOT NULL, " + + "LOCK_ID VARCHAR(%s) NOT NULL, " + + "INSTANCE_ID CHAR(36) NOT NULL, PRIMARY KEY (LOCK_TYPE, LOCK_ID))", + LOCK_TABLE_NAME, LOCK_ID_MAX_LENGTH); + + private static final String CREATE_LOCK_SQL = + String.format( + "INSERT INTO %s (LOCK_TYPE, LOCK_ID, INSTANCE_ID) VALUES (?, ?, ?)", LOCK_TABLE_NAME); + private static final String GET_LOCK_SQL = + String.format("SELECT INSTANCE_ID FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=?", LOCK_TABLE_NAME); + private static final String DELETE_LOCK_SQL = + String.format( + "DELETE FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=? AND INSTANCE_ID=?", LOCK_TABLE_NAME); + + private final String uri; + private final String lockId; + private final Map<String, String> properties; + private transient JdbcClientPool pool; + + /** + * Creates a new {@link TriggerLockFactory}. The lockId should be unique between the users of the + * same uri. + * + * @param uri of the jdbc connection + * @param lockId which should indentify the job and the table + * @param properties used for creating the jdbc connection pool + */ + public JdbcLockFactory(String uri, String lockId, Map<String, String> properties) { + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + Preconditions.checkNotNull(properties, "Properties map is required"); + Preconditions.checkArgument( + lockId.length() < LOCK_ID_MAX_LENGTH, + "Invalid prefix length: lockId should be shorter than %s", + LOCK_ID_MAX_LENGTH); + this.uri = uri; + this.lockId = lockId; + this.properties = properties; + } + + @Override + public void open() { + this.pool = new JdbcClientPool(1, uri, properties); + + if (PropertyUtil.propertyAsBoolean(properties, INIT_LOCK_TABLES_PROPERTY, false)) { + initializeLockTables(); + } + } + + /** Only used in testing to share the jdbc pool */ + @VisibleForTesting + void open(JdbcLockFactory other) { + this.pool = other.pool; + } + + @Override + public Lock createLock() { + return new JdbcLock(pool, lockId, Type.MAINTENANCE); + } + + @Override + public Lock createRecoveryLock() { + return new JdbcLock(pool, lockId, Type.RECOVERY); + } + + @Override + public void close() throws IOException { + pool.close(); + } + + private void initializeLockTables() { + LOG.debug("Creating database tables (if missing) to store table maintenance locks"); + try { + pool.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + LOCK_TABLE_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + LOG.debug("Flink maintenance lock table already exists"); + return true; + } + + LOG.info("Creating Flink maintenance lock table {}", LOCK_TABLE_NAME); + return conn.prepareStatement(CREATE_LOCK_TABLE_SQL).execute(); + }); + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC table maintenance lock"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + private static class JdbcLock implements TriggerLockFactory.Lock { + private final JdbcClientPool pool; + private final String lockId; + private final Type type; + + private JdbcLock(JdbcClientPool pool, String lockId, Type type) { + this.pool = pool; + this.lockId = lockId; + this.type = type; + } + + @Override + public boolean tryLock() { + if (isHeld()) { + LOG.info("Lock is already held for {}", this); + return false; + } + + String newInstanceId = UUID.randomUUID().toString(); + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(CREATE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, newInstanceId); + int count = sql.executeUpdate(); + LOG.info( + "Created {} lock with instanceId {} with row count {}", + this, + newInstanceId, + count); + return count == 1; + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during tryLock"); + } catch (SQLException e) { + // SQL exception happened when creating the lock. Check if the lock creation was + // successful behind the scenes. + if (newInstanceId.equals(instanceId())) { + return true; + } else { + throw new UncheckedSQLException(e, "Failed to create %s lock", this); + } + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public boolean isHeld() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + return rs.next(); + } + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to check the state of the lock %s", this); + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public void unlock() { + try { + // Possible concurrency issue: + // - `unlock` and `tryLock` happens at the same time when there is an existing lock + // + // Steps: + // 1. `unlock` removes the lock in the database, but there is a temporary connection failure + // 2. `lock` founds that there is no lock, so creates a new lock + // 3. `unlock` retires the lock removal and removes the new lock + // + // To prevent the situation above we fetch the current lockId, and remove the lock + // only with the given id. + String instanceId = instanceId(); + + if (instanceId != null) { + pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(DELETE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, instanceId); + long count = sql.executeUpdate(); + LOG.info( + "Deleted {} lock with instanceId {} with row count {}", + this, + instanceId, + count); + } catch (SQLException e) { + // SQL exception happened when deleting lock information + throw new UncheckedSQLException( + e, "Failed to delete %s lock with instanceId %s", this, instanceId); + } + + return null; + }); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + // SQL exception happened when getting/updating lock information + throw new UncheckedSQLException(e, "Failed to remove lock %s", this); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("type", type).add("lockId", lockId).toString(); + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + private String instanceId() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + if (rs.next()) { + return rs.getString(1); + } else { + return null; + } + } + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + } + } + + private enum Type { + MAINTENANCE("m"), + RECOVERY("r"); + + private final String key; + + Type(String key) { + this.key = key; + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java new file mode 100644 index 000000000000..89efffa15f16 --- /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 +class MonitorSource extends SingleThreadedIteratorSource<TableChange> { + 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 + */ + 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<TableChange> getProducedType() { + return TypeInformation.of(TableChange.class); + } + + @Override + Iterator<TableChange> createIterator() { + return new TableChangeIterator(tableLoader, null, maxReadBack); + } + + @Override + SimpleVersionedSerializer<Iterator<TableChange>> iteratorSerializer() { + return new TableChangeIteratorSerializer(tableLoader, maxReadBack); + } + + @Override + public SourceReader<TableChange, GlobalSplit<TableChange>> 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<TableChange> { + 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<Iterator<TableChange>> { + + 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<TableChange> 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 <T> The return type of the source + */ +@Internal +public abstract class SingleThreadedIteratorSource<T> + implements Source< + T, + SingleThreadedIteratorSource.GlobalSplit<T>, + Collection<SingleThreadedIteratorSource.GlobalSplit<T>>>, + ResultTypeQueryable<T> { + 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<T> createIterator(); + + /** + * Serializes the iterator, which is used to save and restore the state of the source. + * + * @return serializer for the iterator + */ + abstract SimpleVersionedSerializer<Iterator<T>> iteratorSerializer(); + + @Override + public SplitEnumerator<GlobalSplit<T>, Collection<GlobalSplit<T>>> createEnumerator( + SplitEnumeratorContext<GlobalSplit<T>> enumContext) { + Preconditions.checkArgument(enumContext.currentParallelism() == 1, PARALLELISM_ERROR); + return new IteratorSourceEnumerator<>( + enumContext, ImmutableList.of(new GlobalSplit<>(createIterator()))); + } + + @Override + public SplitEnumerator<GlobalSplit<T>, Collection<GlobalSplit<T>>> restoreEnumerator( + SplitEnumeratorContext<GlobalSplit<T>> enumContext, Collection<GlobalSplit<T>> checkpoint) { + Preconditions.checkArgument(enumContext.currentParallelism() == 1, PARALLELISM_ERROR); + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer<GlobalSplit<T>> getSplitSerializer() { + return new SplitSerializer<>(iteratorSerializer()); + } + + @Override + public SimpleVersionedSerializer<Collection<GlobalSplit<T>>> getEnumeratorCheckpointSerializer() { + return new EnumeratorSerializer<>(iteratorSerializer()); + } + + @Override + public SourceReader<T, GlobalSplit<T>> 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<T> implements IteratorSourceSplit<T, Iterator<T>> { + private final Iterator<T> iterator; + + GlobalSplit(Iterator<T> iterator) { + this.iterator = iterator; + } + + @Override + public String splitId() { + return "1"; + } + + @Override + public Iterator<T> getIterator() { + return iterator; + } + + @Override + public IteratorSourceSplit<T, Iterator<T>> getUpdatedSplitForIterator( + final Iterator<T> newIterator) { + return new GlobalSplit<>(newIterator); + } + + @Override + public String toString() { + return String.format("GlobalSplit (%s)", iterator); + } + } + + private static final class SplitSerializer<T> + implements SimpleVersionedSerializer<GlobalSplit<T>> { + private final SimpleVersionedSerializer<Iterator<T>> iteratorSerializer; + + SplitSerializer(SimpleVersionedSerializer<Iterator<T>> iteratorSerializer) { + this.iteratorSerializer = iteratorSerializer; + } + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(GlobalSplit<T> split) throws IOException { + return iteratorSerializer.serialize(split.iterator); + } + + @Override + public GlobalSplit<T> deserialize(int version, byte[] serialized) throws IOException { + return new GlobalSplit<>(iteratorSerializer.deserialize(version, serialized)); + } + } + + private static final class EnumeratorSerializer<T> + implements SimpleVersionedSerializer<Collection<GlobalSplit<T>>> { + private static final int CURRENT_VERSION = 1; + private final SimpleVersionedSerializer<Iterator<T>> iteratorSerializer; + + EnumeratorSerializer(SimpleVersionedSerializer<Iterator<T>> iteratorSerializer) { + this.iteratorSerializer = iteratorSerializer; + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(Collection<GlobalSplit<T>> 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<GlobalSplit<T>> 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..773b34b6c495 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.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 dataFileCount; + private long dataFileSizeInBytes; + private int posDeleteFileCount; + private long posDeleteRecordCount; + private int eqDeleteFileCount; + private long eqDeleteRecordCount; + private int commitCount; + + TableChange( + int dataFileCount, + long dataFileSizeInBytes, + int posDeleteFileCount, + long posDeleteRecordCount, + int eqDeleteFileCount, + long eqDeleteRecordCount, + int commitCount) { + this.dataFileCount = dataFileCount; + this.dataFileSizeInBytes = dataFileSizeInBytes; + this.posDeleteFileCount = posDeleteFileCount; + this.posDeleteRecordCount = posDeleteRecordCount; + this.eqDeleteFileCount = eqDeleteFileCount; + this.eqDeleteRecordCount = eqDeleteRecordCount; + this.commitCount = commitCount; + } + + TableChange(Snapshot snapshot, FileIO io) { + Iterable<DataFile> dataFiles = snapshot.addedDataFiles(io); + Iterable<DeleteFile> deleteFiles = snapshot.addedDeleteFiles(io); + + dataFiles.forEach( + dataFile -> { + this.dataFileCount++; + this.dataFileSizeInBytes += dataFile.fileSizeInBytes(); + }); + + deleteFiles.forEach( + deleteFile -> { + switch (deleteFile.content()) { + case POSITION_DELETES: + this.posDeleteFileCount++; + this.posDeleteRecordCount += deleteFile.recordCount(); + break; + case EQUALITY_DELETES: + this.eqDeleteFileCount++; + this.eqDeleteRecordCount += deleteFile.recordCount(); + break; + default: + throw new IllegalArgumentException("Unexpected delete file content: " + deleteFile); + } + }); + + this.commitCount = 1; + } + + static TableChange empty() { + return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); + } + + static Builder builder() { + return new Builder(); + } + + int dataFileCount() { + return dataFileCount; + } + + long dataFileSizeInBytes() { + return dataFileSizeInBytes; + } + + int posDeleteFileCount() { + return posDeleteFileCount; + } + + long posDeleteRecordCount() { + return posDeleteRecordCount; + } + + int eqDeleteFileCount() { + return eqDeleteFileCount; + } + + long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public int commitCount() { + return commitCount; + } + + public void merge(TableChange other) { + this.dataFileCount += other.dataFileCount; + this.dataFileSizeInBytes += other.dataFileSizeInBytes; + this.posDeleteFileCount += other.posDeleteFileCount; + this.posDeleteRecordCount += other.posDeleteRecordCount; + this.eqDeleteFileCount += other.eqDeleteFileCount; + this.eqDeleteRecordCount += other.eqDeleteRecordCount; + this.commitCount += other.commitCount; + } + + TableChange copy() { + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFileCount", dataFileCount) + .add("dataFileSizeInBytes", dataFileSizeInBytes) + .add("posDeleteFileCount", posDeleteFileCount) + .add("posDeleteRecordCount", posDeleteRecordCount) + .add("eqDeleteFileCount", eqDeleteFileCount) + .add("eqDeleteRecordCount", eqDeleteRecordCount) + .add("commitCount", commitCount) + .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 dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes + && posDeleteFileCount == that.posDeleteFileCount + && posDeleteRecordCount == that.posDeleteRecordCount + && eqDeleteFileCount == that.eqDeleteFileCount + && eqDeleteRecordCount == that.eqDeleteRecordCount + && commitCount == that.commitCount; + } + + @Override + public int hashCode() { + return Objects.hash( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } + + static class Builder { + private int dataFileCount = 0; + private long dataFileSizeInBytes = 0L; + private int posDeleteFileCount = 0; + private long posDeleteRecordCount = 0L; + private int eqDeleteFileCount = 0; + private long eqDeleteRecordCount = 0L; + private int commitCount = 0; + + private Builder() {} + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + public TableChange build() { + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java new file mode 100644 index 000000000000..ec0fd920c34b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +public class TableMaintenanceMetrics { + public static final String GROUP_KEY = "maintenanceTask"; + public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + + // TriggerManager metrics + public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; + public static final String CONCURRENT_RUN_THROTTLED = "concurrentRunThrottled"; + public static final String TRIGGERED = "triggered"; + public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + + private TableMaintenanceMetrics() { + // do not instantiate + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java new file mode 100644 index 000000000000..85c6c8dbdd55 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +@Internal +class Trigger { + private final long timestamp; + private final SerializableTable table; + private final Integer taskId; + private final boolean isRecovery; + + private Trigger(long timestamp, SerializableTable table, Integer taskId, boolean isRecovery) { + this.timestamp = timestamp; + this.table = table; + this.taskId = taskId; + this.isRecovery = isRecovery; + } + + static Trigger create(long timestamp, SerializableTable table, int taskId) { + return new Trigger(timestamp, table, taskId, false); + } + + static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, null, true); + } + + long timestamp() { + return timestamp; + } + + SerializableTable table() { + return table; + } + + Integer taskId() { + return taskId; + } + + boolean isRecovery() { + return isRecovery; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("timestamp", timestamp) + .add("table", table == null ? null : table.name()) + .add("taskId", taskId) + .add("isRecovery", isRecovery) + .toString(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java new file mode 100644 index 000000000000..dba33b22a42a --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Serializable; +import java.time.Duration; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TriggerEvaluator implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); + private final List<Predicate> predicates; + + private TriggerEvaluator(List<Predicate> predicates) { + Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 condition."); + + this.predicates = predicates; + } + + boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { + boolean result = + predicates.stream().anyMatch(p -> p.evaluate(event, lastTimeMs, currentTimeMs)); + LOG.debug( + "Checking event: {}, at {}, last: {} with result: {}", + event, + currentTimeMs, + lastTimeMs, + result); + return result; + } + + static class Builder implements Serializable { + private Integer dataFileCount; + private Long dataFileSizeInBytes; + private Integer posDeleteFileCount; + private Long posDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer commitCount; + private Duration timeout; + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long neDataFileSizeInBytes) { + this.dataFileSizeInBytes = neDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; + return this; + } + + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; + return this; + } + + Builder timeout(Duration newTimeout) { + this.timeout = newTimeout; + return this; + } + + TriggerEvaluator build() { + List<Predicate> predicates = Lists.newArrayList(); + if (dataFileCount != null) { + predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); + } + + if (dataFileSizeInBytes != null) { + predicates.add( + (change, unused, unused2) -> change.dataFileSizeInBytes() >= dataFileSizeInBytes); + } + + if (posDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteFileCount() >= posDeleteFileCount); + } + + if (posDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteRecordCount() >= posDeleteRecordCount); + } + + if (eqDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteFileCount() >= eqDeleteFileCount); + } + + if (eqDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.eqDeleteRecordCount() >= eqDeleteRecordCount); + } + + if (commitCount != null) { + predicates.add((change, unused, unused2) -> change.commitCount() >= commitCount); + } + + if (timeout != null) { + predicates.add( + (change, lastTimeMs, currentTimeMs) -> + currentTimeMs - lastTimeMs >= timeout.toMillis()); + } + + return new TriggerEvaluator(predicates); + } + } + + private interface Predicate extends Serializable { + boolean evaluate(TableChange event, long lastTimeMs, long currentTimeMs); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java new file mode 100644 index 000000000000..329223d27ccf --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.Closeable; +import java.io.Serializable; +import org.apache.flink.annotation.Experimental; + +/** Lock interface for handling locks for the Flink Table Maintenance jobs. */ +@Experimental +public interface TriggerLockFactory extends Serializable, Closeable { + void open(); + + Lock createLock(); + + Lock createRecoveryLock(); + + interface Lock { + /** + * Tries to acquire a lock with a given key. Anyone already holding a lock would prevent + * acquiring this lock. Not reentrant. + * + * <p>Called by {@link TriggerManager}. Implementations could assume that are no concurrent + * calls for this method. + * + * @return <code>true</code> if the lock is acquired by this job, <code>false</code> if the lock + * is already held by someone + */ + boolean tryLock(); + + /** + * Checks if the lock is already taken. + * + * @return <code>true</code> if the lock is held by someone + */ + boolean isHeld(); + + // TODO: Fix the link to the LockRemover when we have a final name and implementation + /** + * Releases the lock. Should not fail if the lock is not held by anyone. + * + * <p>Called by LockRemover. Implementations could assume that are no concurrent calls for this + * method. + */ + void unlock(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java new file mode 100644 index 000000000000..f4c3c1d47cf7 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -0,0 +1,339 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.util.Collector; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} messages which are + * calculated based on the incoming {@link TableChange} messages. The TriggerManager keeps track of + * the changes since the last run of the Maintenance Tasks and triggers a new run based on the + * result of the {@link TriggerEvaluator}. + * + * <p>The TriggerManager prevents overlapping Maintenance Task runs using {@link + * TriggerLockFactory.Lock}. The current implementation only handles conflicts within a single job. + * Users should avoid scheduling maintenance for the same table in different Flink jobs. + * + * <p>The TriggerManager should run as a global operator. {@link KeyedProcessFunction} is used, so + * the timer functions are available, but the key is not used. + */ +@Internal +class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, Trigger> + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); + + private final TableLoader tableLoader; + private final TriggerLockFactory lockFactory; + private final List<String> taskNames; + private final List<TriggerEvaluator> evaluators; + private final long minFireDelayMs; + private final long lockCheckDelayMs; + private transient Counter rateLimiterTriggeredCounter; + private transient Counter concurrentRunThrottledCounter; + private transient Counter nothingToTriggerCounter; + private transient List<Counter> triggerCounters; + private transient ValueState<Long> nextEvaluationTimeState; + private transient ListState<TableChange> accumulatedChangesState; + private transient ListState<Long> lastTriggerTimesState; + private transient Long nextEvaluationTime; + private transient List<TableChange> accumulatedChanges; + private transient List<Long> lastTriggerTimes; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient boolean shouldRestoreTasks = false; + private transient boolean inited = false; + // To keep the task scheduling fair we keep the last triggered task position in memory. + // If we find a task to trigger, then we run it, but after it is finished, we start from the given + // position to prevent "starvation" of the tasks. + // When there is nothing to trigger, we start from the beginning, as the order of the tasks might + // be important (RewriteDataFiles first, and then RewriteManifestFiles later) + private transient int startsFrom = 0; + private transient boolean triggered = false; + + TriggerManager( + TableLoader tableLoader, + TriggerLockFactory lockFactory, + List<String> taskNames, + List<TriggerEvaluator> evaluators, + long minFireDelayMs, + long lockCheckDelayMs) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + Preconditions.checkArgument( + evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); + Preconditions.checkArgument( + taskNames.size() == evaluators.size(), "Provide a name and evaluator for all of the tasks"); + Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); + Preconditions.checkArgument( + lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); + + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskNames = taskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.rateLimiterTriggeredCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); + this.concurrentRunThrottledCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); + this.nothingToTriggerCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); + this.triggerCounters = + taskNames.stream() + .map( + name -> + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.TRIGGERED)) + .collect(Collectors.toList()); + + this.nextEvaluationTimeState = + getRuntimeContext() + .getState(new ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG)); + this.accumulatedChangesState = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>( + "triggerManagerAccumulatedChange", TypeInformation.of(TableChange.class))); + this.lastTriggerTimesState = + getRuntimeContext() + .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); + + tableLoader.open(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (inited) { + // Only store state if initialized + nextEvaluationTimeState.update(nextEvaluationTime); + accumulatedChangesState.update(accumulatedChanges); + lastTriggerTimesState.update(lastTriggerTimes); + LOG.info( + "Storing state: nextEvaluationTime {}, accumulatedChanges {}, lastTriggerTimes {}", + nextEvaluationTime, + accumulatedChanges, + lastTriggerTimes); + } else { + LOG.info("Not initialized, state is not stored"); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOG.info("Initializing state restored: {}", context.isRestored()); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + if (context.isRestored()) { + shouldRestoreTasks = true; + } + } + + @Override + public void processElement(TableChange change, Context ctx, Collector<Trigger> out) + throws Exception { + init(out, ctx.timerService()); + + accumulatedChanges.forEach(tableChange -> tableChange.merge(change)); + + long current = ctx.timerService().currentProcessingTime(); + if (nextEvaluationTime == null) { + checkAndFire(current, ctx.timerService(), out); + } else { + LOG.info( + "Trigger manager rate limiter triggered current: {}, next: {}, accumulated changes: {}", + current, + nextEvaluationTime, + accumulatedChanges); + rateLimiterTriggeredCounter.inc(); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> out) throws Exception { + init(out, ctx.timerService()); + this.nextEvaluationTime = null; + checkAndFire(ctx.timerService().currentProcessingTime(), ctx.timerService(), out); + } + + @Override + public void close() throws IOException { + tableLoader.close(); + lockFactory.close(); + } + + private void checkAndFire(long current, TimerService timerService, Collector<Trigger> out) { + if (shouldRestoreTasks) { + if (recoveryLock.isHeld()) { + // Recovered tasks in progress. Skip trigger check + LOG.debug("The recovery lock is still held at {}", current); + schedule(timerService, current + lockCheckDelayMs); + return; + } else { + LOG.info("The recovery is finished at {}", current); + shouldRestoreTasks = false; + } + } + + Integer taskToStart = + nextTrigger(evaluators, accumulatedChanges, lastTriggerTimes, current, startsFrom); + if (taskToStart == null) { + // Nothing to execute + if (!triggered) { + nothingToTriggerCounter.inc(); + LOG.debug("Nothing to execute at {} for collected: {}", current, accumulatedChanges); + } else { + LOG.debug("Execution check finished"); + } + + // Next time start from the beginning + startsFrom = 0; + triggered = false; + return; + } + + if (lock.tryLock()) { + TableChange change = accumulatedChanges.get(taskToStart); + SerializableTable table = + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()); + out.collect(Trigger.create(current, table, taskToStart)); + LOG.debug("Fired event with time: {}, collected: {} for {}", current, change, table.name()); + triggerCounters.get(taskToStart).inc(); + accumulatedChanges.set(taskToStart, TableChange.empty()); + lastTriggerTimes.set(taskToStart, current); + schedule(timerService, current + minFireDelayMs); + startsFrom = (taskToStart + 1) % evaluators.size(); + triggered = true; + } else { + // A task is already running, waiting for it to finish + LOG.info("Failed to acquire lock. Delaying task to {}", current + lockCheckDelayMs); + + startsFrom = taskToStart; + concurrentRunThrottledCounter.inc(); + schedule(timerService, current + lockCheckDelayMs); + } + + timerService.registerProcessingTimeTimer(nextEvaluationTime); + } + + private void schedule(TimerService timerService, long time) { + this.nextEvaluationTime = time; + timerService.registerProcessingTimeTimer(time); + } + + private static Integer nextTrigger( + List<TriggerEvaluator> evaluators, + List<TableChange> changes, + List<Long> lastTriggerTimes, + long currentTime, + int startPos) { + int current = startPos; + do { + if (evaluators + .get(current) + .check(changes.get(current), lastTriggerTimes.get(current), currentTime)) { + return current; + } + + current = (current + 1) % evaluators.size(); + } while (current != startPos); + + return null; + } + + private void init(Collector<Trigger> out, TimerService timerService) throws Exception { + if (!inited) { + long current = timerService.currentProcessingTime(); + + // Initialize from state + this.nextEvaluationTime = nextEvaluationTimeState.value(); + this.accumulatedChanges = Lists.newArrayList(accumulatedChangesState.get()); + this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get()); + + // Initialize if the state was empty + if (accumulatedChanges.isEmpty()) { + for (int i = 0; i < evaluators.size(); ++i) { + accumulatedChanges.add(TableChange.empty()); + lastTriggerTimes.add(current); + } + } + + if (shouldRestoreTasks) { + // When the job state is restored, there could be ongoing tasks. + // To prevent collision with the new triggers the following is done: + // - add a recovery lock + // - fire a recovery trigger + // This ensures that the tasks of the previous trigger are executed, and the lock is removed + // in the end. The result of the 'tryLock' is ignored as an already existing lock prevents + // collisions as well. + recoveryLock.tryLock(); + out.collect(Trigger.recovery(current)); + if (nextEvaluationTime == null) { + schedule(timerService, current + minFireDelayMs); + } + } + + inited = true; + } + } +} diff --git a/flink/v1.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<RowData> newAppender(OutputFile outputFile, FileFormat format) { - return newAppender(EncryptionUtil.plainAsEncryptedOutput(outputFile), format); - } - - @Override - public FileAppender<RowData> newAppender(EncryptedOutputFile outputFile, FileFormat format) { MetricsConfig metricsConfig = MetricsConfig.forTable(table); try { switch (format) { @@ -148,7 +142,7 @@ public FileAppender<RowData> newAppender(EncryptedOutputFile outputFile, FileFor public DataWriter<RowData> 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<RowData> 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<RowData> 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<RowData> 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<RowData> 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<RowData> 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<RowData> 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 82% 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 index c7e8a2dea7cb..9571efdc5268 100644 --- 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 @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List<ManifestFile> manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } diff --git a/flink/v1.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 77% 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 index 769af7d77140..c53431490984 100644 --- 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 @@ -53,15 +53,20 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.shuffle.DataStatisticsOperatorFactory; +import org.apache.iceberg.flink.sink.shuffle.RangePartitioner; +import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -233,15 +238,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + * <p>Default is {@link StatisticsType#Auto} where initially Map statistics is used. But if + * cardinality is higher than the threshold (currently 10K) as defined in {@code + * SketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD}, statistics collection automatically switches to + * the sketch reservoir sampling. + * + * <p>Explicit set the statistics type if the default behavior doesn't work. + * + * @param type to specify the statistics type for range distribution. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rangeDistributionStatisticsType(StatisticsType type) { + if (type != null) { + writeOptions.put(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key(), type.name()); + } + return this; + } + + /** + * If sort order contains partition columns, each sort key would map to one partition and data + * file. This relative weight can avoid placing too many small files for sort keys with low + * traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means + * each key has a base weight of `2%` of the targeted traffic weight per writer task. + * + * <p>E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream + * contains events from now up to 180 days ago. With event time, traffic weight distribution + * across different days typically has a long tail pattern. Current day contains the most + * traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism + * is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer + * task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, + * the range partitioner would put all the oldest 150 days in one writer task. That writer task + * would write to 150 small files (one per day). Keeping 150 open files can potentially consume + * large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time + * can also be potentially slow. If this config is set to `0.02`. It means every sort key has a + * base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially + * avoid placing more than `50` data files (one per day) on one writer task no matter how small + * they are. + * + * <p>This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For + * {@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as + * partition columns. Otherwise, too many partitions and small files may be generated during + * write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. + * + * <p>Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -347,20 +405,23 @@ private <T> DataStreamSink<T> chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List<Integer> equalityFieldIds = checkAndGetEqualityFieldIds(); + List<Integer> equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream<RowData> distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator<WriteResult> writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + SingleOutputStreamOperator<FlinkWriteResult> writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -426,7 +487,7 @@ private <T> DataStreamSink<T> appendDummySink( } private SingleOutputStreamOperator<Void> appendCommitter( - SingleOutputStreamOperator<WriteResult> writerStream) { + SingleOutputStreamOperator<FlinkWriteResult> writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -446,8 +507,11 @@ private SingleOutputStreamOperator<Void> appendCommitter( return committerStream; } - private SingleOutputStreamOperator<WriteResult> appendWriter( - DataStream<RowData> input, RowType flinkRowType, List<Integer> equalityFieldIds) { + private SingleOutputStreamOperator<FlinkWriteResult> appendWriter( + DataStream<RowData> input, + RowType flinkRowType, + List<Integer> equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +545,13 @@ private SingleOutputStreamOperator<WriteResult> appendWriter( IcebergStreamWriter<RowData> streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator<WriteResult> writerStream = + SingleOutputStreamOperator<FlinkWriteResult> writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +561,15 @@ private SingleOutputStreamOperator<WriteResult> appendWriter( private DataStream<RowData> distributeDataStream( DataStream<RowData> input, List<Integer> equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +611,52 @@ private DataStream<RowData> distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator<StatisticsOrRecord> shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +671,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + * <p>{@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + * <p>In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer<IcebergCommittable> { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..2245b36f15a7 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,311 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + * <ul> + * <li>There is a single {@link IcebergCommittable} for every checkpoint + * <li>There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + * <li>There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + * </ul> + */ +class IcebergCommitter implements Committer<IcebergCommittable> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + public static final WriteResult EMPTY_WRITE_RESULT = + WriteResult.builder() + .addDataFiles(Lists.newArrayList()) + .addDeleteFiles(Lists.newArrayList()) + .build(); + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + private final String branch; + private final Map<String, String> snapshotProperties; + private final boolean replacePartitions; + private IcebergFilesCommitterMetrics committerMetrics; + private Table table; + private final TableLoader tableLoader; + private int maxContinuousEmptyCommits; + private ExecutorService workerPool; + private int continuousEmptyCheckpoints = 0; + + IcebergCommitter( + TableLoader tableLoader, + String branch, + Map<String, String> snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + this.tableLoader = tableLoader; + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection<CommitRequest<IcebergCommittable>> commitRequests) + throws IOException, InterruptedException { + if (commitRequests.isEmpty()) { + return; + } + + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap = Maps.newTreeMap(); + for (CommitRequest<IcebergCommittable> request : commitRequests) { + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + long maxCommittedCheckpointId = + SinkUtil.getMaxCommittedCheckpointId(table, last.jobId(), last.operatorId(), branch); + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + NavigableMap<Long, CommitRequest<IcebergCommittable>> uncommitted = + commitRequestMap.tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests(uncommitted, last.jobId(), last.operatorId()); + } + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link + * org.apache.iceberg.flink.sink.DeltaManifests} ordered by the checkpointId, and writing the new + * snapshot to the Iceberg table. The {@link org.apache.iceberg.SnapshotSummary} will contain the + * jobId, snapshotId, checkpointId so in case of job restart we can identify which changes are + * committed, and which are still waiting for the commit. + * + * @param commitRequestMap The checkpointId to {@link CommitRequest} map of the changes to commit + * @param newFlinkJobId The jobId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @param operatorId The operatorId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @throws IOException On commit failure + */ + private void commitPendingRequests( + NavigableMap<Long, CommitRequest<IcebergCommittable>> commitRequestMap, + String newFlinkJobId, + String operatorId) + throws IOException { + long checkpointId = commitRequestMap.lastKey(); + List<ManifestFile> manifests = Lists.newArrayList(); + NavigableMap<Long, WriteResult> pendingResults = Maps.newTreeMap(); + for (Map.Entry<Long, CommitRequest<IcebergCommittable>> e : commitRequestMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue().getCommittable().manifest())) { + pendingResults.put(e.getKey(), EMPTY_WRITE_RESULT); + } else { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue().getCommittable().manifest()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(summary); + } + + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void logCommitSummary(CommitSummary summary, String description) { + LOG.info( + "Preparing for commit: {} on table: {} branch: {} with summary: {}.", + description, + table, + branch, + summary); + } + + private void commitPendingResult( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId); + } + continuousEmptyCheckpoints = 0; + } else { + long checkpointId = pendingResults.lastKey(); + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void replacePartitions( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + String description = "dynamic partition overwrite"; + + logCommitSummary(summary, description); + commitOperation(dynamicOverwrite, description, newFlinkJobId, operatorId, checkpointId); + } + + private void commitDeltaTxn( + NavigableMap<Long, WriteResult> pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + String description = "append"; + logCommitSummary(summary, description); + // fail all commits as really its only one + commitOperation(appendFiles, description, newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry<Long, WriteResult> e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + + String description = "rowDelta"; + logCommitSummary(summary, description); + commitOperation(rowDelta, description, newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate<?> operation, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(SinkUtil.MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(SinkUtil.FLINK_JOB_ID, newFlinkJobId); + operation.set(SinkUtil.OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + if (committerMetrics != null) { + committerMetrics.commitDuration(durationMs); + } + } + + @Override + public void close() throws IOException { + tableLoader.close(); + } +} diff --git a/flink/v1.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 87% 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 index b9bceaa9311d..609deb621f43 100644 --- 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 @@ -44,13 +44,11 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -63,7 +61,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator<Void> - implements OneInputStreamOperator<WriteResult, Void>, BoundedOneInput { + implements OneInputStreamOperator<FlinkWriteResult, Void>, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -96,7 +94,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator<Void> // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List<WriteResult> writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map<Long, List<WriteResult>> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -187,7 +185,7 @@ public void initializeState(StateInitializationContext context) throws Exception // it's safe to assign the max committed checkpoint id from restored flink job to the current // flink job. this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + SinkUtil.getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); NavigableMap<Long, byte[]> uncommittedDataFiles = Maps.newTreeMap(checkpointsState.get().iterator().next()) @@ -212,7 +210,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -220,8 +219,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -280,7 +277,7 @@ private void commitUpToCheckpoint( commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); committerMetrics.updateCommitSummary(summary); pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); } private void commitPendingResult( @@ -303,27 +300,6 @@ private void commitPendingResult( } } - private void deleteCommittedManifests( - List<ManifestFile> manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - private void replacePartitions( NavigableMap<Long, WriteResult> pendingResults, CommitSummary summary, @@ -426,30 +402,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord<WriteResult> element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord<FlinkWriteResult> element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List<WriteResult> writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry<Long, List<WriteResult>> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List<WriteResult> writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); @@ -489,28 +480,4 @@ static ListStateDescriptor<SortedMap<Long, byte[]>> buildStateDescriptor() { longComparator); return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map<String, String> summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.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<Long> { - 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.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java new file mode 100644 index 000000000000..d080169544cd --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -0,0 +1,742 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. We will use the + * following: + * + * <ul> + * <li>{@link SupportsPreWriteTopology} which redistributes the data to the writers based on the + * {@link DistributionMode} + * <li>{@link org.apache.flink.api.connector.sink2.SinkWriter} which writes data/delete files, and + * generates the {@link org.apache.iceberg.io.WriteResult} objects for the files + * <li>{@link SupportsPreCommitTopology} which we use to place the {@link + * org.apache.iceberg.flink.sink.IcebergWriteAggregator} which merges the individual {@link + * org.apache.flink.api.connector.sink2.SinkWriter}'s {@link + * org.apache.iceberg.io.WriteResult}s to a single {@link + * org.apache.iceberg.flink.sink.IcebergCommittable} + * <li>{@link org.apache.iceberg.flink.sink.IcebergCommitter} which commits the incoming{@link + * org.apache.iceberg.flink.sink.IcebergCommittable}s to the Iceberg table + * <li>{@link SupportsPostCommitTopology} we could use for incremental compaction later. This is + * not implemented yet. + * </ul> + * + * The job graph looks like below: + * + * <pre>{@code + * Flink sink + * +-----------------------------------------------------------------------------------+ + * | | + * +-------+ | +----------+ +-------------+ +---------------+ | + * | Map 1 | ==> | | writer 1 | | committer 1 | ---> | post commit 1 | | + * +-------+ | +----------+ +-------------+ +---------------+ | + * | \ / \ | + * | \ / \ | + * | \ / \ | + * +-------+ | +----------+ \ +-------------------+ / +-------------+ \ +---------------+ | + * | Map 2 | ==> | | writer 2 | --->| commit aggregator | | committer 2 | | post commit 2 | | + * +-------+ | +----------+ +-------------------+ +-------------+ +---------------+ | + * | Commit only on | + * | committer 1 | + * +-----------------------------------------------------------------------------------+ + * }</pre> + */ +@Experimental +public class IcebergSink + implements Sink<RowData>, + SupportsPreWriteTopology<RowData>, + SupportsCommitter<IcebergCommittable>, + SupportsPreCommitTopology<WriteResult, IcebergCommittable>, + SupportsPostCommitTopology<IcebergCommittable> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + private final TableLoader tableLoader; + private final Map<String, String> snapshotProperties; + private final String uidSuffix; + private final String sinkId; + private final Map<String, String> writeProperties; + private final RowType flinkRowType; + private final SerializableSupplier<Table> tableSupplier; + private final transient FlinkWriteConf flinkWriteConf; + private final List<Integer> equalityFieldIds; + private final boolean upsertMode; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final String branch; + private final boolean overwriteMode; + private final int workerPoolSize; + + private final Table table; + private final List<String> equalityFieldColumns = null; + + private IcebergSink( + TableLoader tableLoader, + Table table, + Map<String, String> snapshotProperties, + String uidSuffix, + Map<String, String> writeProperties, + RowType flinkRowType, + SerializableSupplier<Table> tableSupplier, + FlinkWriteConf flinkWriteConf, + List<Integer> equalityFieldIds, + String branch, + boolean overwriteMode) { + this.tableLoader = tableLoader; + this.snapshotProperties = snapshotProperties; + this.uidSuffix = uidSuffix; + this.writeProperties = writeProperties; + this.flinkRowType = flinkRowType; + this.tableSupplier = tableSupplier; + this.flinkWriteConf = flinkWriteConf; + this.equalityFieldIds = equalityFieldIds; + this.branch = branch; + this.overwriteMode = overwriteMode; + this.table = table; + this.upsertMode = flinkWriteConf.upsertMode(); + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter<RowData> createWriter(InitContext context) { + RowDataTaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + targetDataFileSize, + dataFileFormat, + writeProperties, + equalityFieldIds, + upsertMode); + IcebergStreamWriterMetrics metrics = + new IcebergStreamWriterMetrics(context.metricGroup(), table.name()); + return new IcebergSinkWriter( + tableSupplier.get().name(), + taskWriterFactory, + metrics, + context.getSubtaskId(), + context.getAttemptNumber()); + } + + @Override + public Committer<IcebergCommittable> createCommitter(CommitterInitContext context) { + IcebergFilesCommitterMetrics metrics = + new IcebergFilesCommitterMetrics(context.metricGroup(), table.name()); + return new IcebergCommitter( + tableLoader, branch, snapshotProperties, overwriteMode, workerPoolSize, sinkId, metrics); + } + + @Override + public SimpleVersionedSerializer<IcebergCommittable> getCommittableSerializer() { + return new IcebergCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream<CommittableMessage<IcebergCommittable>> committables) { + // TODO Support small file compaction + } + + @Override + public DataStream<RowData> addPreWriteTopology(DataStream<RowData> inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream<CommittableMessage<IcebergCommittable>> addPreCommitTopology( + DataStream<CommittableMessage<WriteResult>> writeResults) { + TypeInformation<CommittableMessage<IcebergCommittable>> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + String suffix = defaultSuffix(uidSuffix, table.name()); + String preCommitAggregatorUid = String.format("Sink pre-commit aggregator: %s", suffix); + + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + return writeResults + .global() + .transform(preCommitAggregatorUid, typeInformation, new IcebergWriteAggregator(tableLoader)) + .uid(preCommitAggregatorUid) + .setParallelism(1) + .setMaxParallelism(1) + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + .global(); + } + + @Override + public SimpleVersionedSerializer<WriteResult> getWriteResultSerializer() { + return new WriteResultSerializer(); + } + + public static class Builder { + private TableLoader tableLoader; + private String uidSuffix = ""; + private Function<String, DataStream<RowData>> inputCreator = null; + private TableSchema tableSchema; + private SerializableTable table; + private final Map<String, String> writeOptions = Maps.newHashMap(); + private final Map<String, String> snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private List<String> equalityFieldColumns = null; + + private Builder() {} + + private Builder forRowData(DataStream<RowData> newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forRow(DataStream<Row> input, TableSchema inputTableSchema) { + RowType rowType = (RowType) inputTableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = inputTableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return forMapperOutputType( + input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(inputTableSchema); + } + + private <T> Builder forMapperOutputType( + DataStream<T> input, MapFunction<T, RowData> mapper, TypeInformation<RowData> outputType) { + this.inputCreator = + newUidSuffix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism of map operator same as its input to keep map operator + // chaining its input, and avoid rebalanced by default. + SingleOutputStreamOperator<RowData> inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidSuffix != null) { + String uid = String.format("Sink pre-writer mapper: %s", newUidSuffix); + inputStream.name(uid).uid(uid); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link SerializableTable} instance is used for initializing {@link + * IcebergStreamWriter} which will write all the records into {@link DataFile}s and emit them to + * downstream operator. Providing a table would avoid so many table loading from each separate + * task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = (SerializableTable) SerializableTable.copyOf(newTable); + return this; + } + + /** + * The table loader is used for loading tables in {@link + * org.apache.iceberg.flink.sink.IcebergCommitter} lazily, we need this loader because {@link + * Table} is not serializable and could not just use the loaded table from Builder#table in the + * remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + TableLoader tableLoader() { + return tableLoader; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map<String, String> properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the IcebergSink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List<String> columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid suffix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator). Actual operator uid will be appended + * with a suffix like "Sink Committer: $uidSuffix". + * + * <p>Flink auto generates operator uid if not set explicitly. It is a recommended <a + * href="https://ci.apache.org/projects/flink/flink-docs-master/docs/ops/production_ready/"> + * best-practice to set uid for all operators</a> before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid. + * + * <p>Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newSuffix suffix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidSuffix(String newSuffix) { + this.uidSuffix = newSuffix; + return this; + } + + public Builder snapshotProperties(Map<String, String> properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + IcebergSink build() { + + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader(), "Table loader shouldn't be null"); + + // Set the table if it is not yet set in the builder, so we can do the equalityId checks + SerializableTable serializableTable = checkAndGetTable(tableLoader(), table); + this.table = serializableTable; + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + SerializableSupplier<Table> tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List<Integer> equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink<RowData> append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream<RowData> rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: <uidSuffix> + DataStreamSink<RowData> rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map<String, String> writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map<String, String> writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream<RowData> distributeDataStream(DataStream<RowData> input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param <T> the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static <T> Builder builderFor( + DataStream<T> input, MapFunction<T, RowData> mapper, TypeInformation<RowData> outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream<RowData> input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter<RowData, WriteResult> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory<RowData> taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter<RowData> writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory<RowData> taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection<WriteResult> prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.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 88% 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 index 9ea0349fb057..412d6c7081bf 100644 --- 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 @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter<T> extends AbstractStreamOperator<WriteResult> - implements OneInputStreamOperator<T, WriteResult>, BoundedOneInput { +class IcebergStreamWriter<T> extends AbstractStreamOperator<FlinkWriteResult> + implements OneInputStreamOperator<T, FlinkWriteResult>, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory<T> taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,20 +90,20 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.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.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator<CommittableMessage<IcebergCommittable>> + implements OneInputStreamOperator< + CommittableMessage<WriteResult>, CommittableMessage<IcebergCommittable>> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection<WriteResult> results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage<IcebergCommittable> summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage<IcebergCommittable> message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection<WriteResult> writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord<CommittableMessage<WriteResult>> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage<WriteResult>) element.getValue()).getCommittable()); + } + } +} 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 94% 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 index da5e6e7627ae..1cb7f4dea1e8 100644 --- 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 @@ -26,13 +26,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier<Table> tableSupplier; private final Map<String, String> props; private final String flinkJobId; 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.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String OPERATOR_ID = "flink.operator-id"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List<Integer> checkAndGetEqualityFieldIds(Table table, List<String> equalityFieldColumns) { + List<Integer> equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set<Integer> equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map<String, String> summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/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.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +class WriteResultSerializer implements SimpleVersionedSerializer<WriteResult> { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.20/flink/src/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 new file mode 100644 index 000000000000..5525f02c873e --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.datasketches.sampling.ReservoirItemsUnion; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. + */ +class AggregatedStatisticsTracker { + private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); + + private final String operatorName; + private final int parallelism; + private final TypeSerializer<DataStatistics> statisticsSerializer; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final int switchToSketchThreshold; + private final NavigableMap<Long, Aggregation> aggregationsPerCheckpoint; + + private CompletedStatistics completedStatistics; + + AggregatedStatisticsTracker( + String operatorName, + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable CompletedStatistics restoredStatistics) { + this.operatorName = operatorName; + this.parallelism = parallelism; + this.statisticsSerializer = + new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.switchToSketchThreshold = switchToSketchThreshold; + this.completedStatistics = restoredStatistics; + + this.aggregationsPerCheckpoint = Maps.newTreeMap(); + } + + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { + long checkpointId = event.checkpointId(); + LOG.debug( + "Handling statistics event from subtask {} of operator {} for checkpoint {}", + subtask, + operatorName, + checkpointId); + + if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { + LOG.info( + "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " + + "Was expecting data statistics from checkpoint higher than {}", + operatorName, + subtask, + checkpointId, + completedStatistics.checkpointId()); + return null; + } + + Aggregation aggregation = + aggregationsPerCheckpoint.computeIfAbsent( + checkpointId, + ignored -> + new Aggregation( + parallelism, + downstreamParallelism, + switchToSketchThreshold, + statisticsType, + StatisticsUtil.collectType(statisticsType, completedStatistics))); + DataStatistics dataStatistics = + StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); + if (!aggregation.merge(subtask, dataStatistics)) { + LOG.debug( + "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } + + if (aggregation.isComplete()) { + this.completedStatistics = aggregation.completedStatistics(checkpointId); + // clean up aggregations up to the completed checkpoint id + aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); + return completedStatistics; + } + + return null; + } + + @VisibleForTesting + NavigableMap<Long, Aggregation> aggregationsPerCheckpoint() { + return aggregationsPerCheckpoint; + } + + static class Aggregation { + private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); + + private final Set<Integer> subtaskSet; + private final int parallelism; + private final int downstreamParallelism; + private final int switchToSketchThreshold; + private final StatisticsType configuredType; + private StatisticsType currentType; + private Map<SortKey, Long> mapStatistics; + private ReservoirItemsUnion<SortKey> sketchStatistics; + + Aggregation( + int parallelism, + int downstreamParallelism, + int switchToSketchThreshold, + StatisticsType configuredType, + StatisticsType currentType) { + this.subtaskSet = Sets.newHashSet(); + this.parallelism = parallelism; + this.downstreamParallelism = downstreamParallelism; + this.switchToSketchThreshold = switchToSketchThreshold; + this.configuredType = configuredType; + this.currentType = currentType; + + if (currentType == StatisticsType.Map) { + this.mapStatistics = Maps.newHashMap(); + this.sketchStatistics = null; + } else { + this.mapStatistics = null; + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + } + } + + @VisibleForTesting + Set<Integer> subtaskSet() { + return subtaskSet; + } + + @VisibleForTesting + StatisticsType currentType() { + return currentType; + } + + @VisibleForTesting + Map<SortKey, Long> mapStatistics() { + return mapStatistics; + } + + @VisibleForTesting + ReservoirItemsUnion<SortKey> sketchStatistics() { + return sketchStatistics; + } + + private boolean isComplete() { + return subtaskSet.size() == parallelism; + } + + /** @return false if duplicate */ + private boolean merge(int subtask, DataStatistics taskStatistics) { + if (subtaskSet.contains(subtask)) { + return false; + } + + subtaskSet.add(subtask); + merge(taskStatistics); + return true; + } + + @SuppressWarnings("unchecked") + private void merge(DataStatistics taskStatistics) { + if (taskStatistics.type() == StatisticsType.Map) { + Map<SortKey, Long> taskMapStats = (Map<SortKey, Long>) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); + if (configuredType == StatisticsType.Auto + && mapStatistics.size() > switchToSketchThreshold) { + convertCoordinatorToSketch(); + } + } else { + // convert task stats to sketch first + ReservoirItemsSketch<SortKey> taskSketch = + ReservoirItemsSketch.newInstance( + SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); + SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); + sketchStatistics.update(taskSketch); + } + } else { + ReservoirItemsSketch<SortKey> taskSketch = + (ReservoirItemsSketch<SortKey>) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + // convert global stats to sketch first + convertCoordinatorToSketch(); + } + + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } + } + } + + private void convertCoordinatorToSketch() { + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); + this.currentType = StatisticsType.Sketch; + this.mapStatistics = null; + } + + private CompletedStatistics completedStatistics(long checkpointId) { + if (currentType == StatisticsType.Map) { + LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + } else { + ReservoirItemsSketch<SortKey> sketch = sketchStatistics.getResult(); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } + } + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java new file mode 100644 index 000000000000..e4cba174f0f2 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). + */ +class CompletedStatistics { + private final long checkpointId; + private final StatisticsType type; + private final Map<SortKey, Long> keyFrequency; + private final SortKey[] keySamples; + + static CompletedStatistics fromKeyFrequency(long checkpointId, Map<SortKey, Long> stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( + long checkpointId, + StatisticsType type, + Map<SortKey, Long> keyFrequency, + SortKey[] keySamples) { + this.checkpointId = checkpointId; + this.type = type; + this.keyFrequency = keyFrequency; + this.keySamples = keySamples; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("keyFrequency", keyFrequency) + .add("keySamples", keySamples) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof CompletedStatistics)) { + return false; + } + + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(keyFrequency, other.keyFrequency()) + && Arrays.equals(keySamples, other.keySamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + Map<SortKey, Long> keyFrequency() { + return keyFrequency; + } + + SortKey[] keySamples() { + return keySamples; + } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java new file mode 100644 index 000000000000..1ac0e386a011 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; + +class CompletedStatisticsSerializer extends TypeSerializer<CompletedStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final MapSerializer<SortKey, Long> keyFrequencySerializer; + private final ListSerializer<SortKey> keySamplesSerializer; + + CompletedStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<CompletedStatistics> duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); + } + + @Override + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + keyFrequencySerializer.serialize(record.keyFrequency(), target); + } else { + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); + } + } + + @Override + public CompletedStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + Map<SortKey, Long> keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); + } else { + List<SortKey> sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); + } + } + + @Override + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<CompletedStatistics> snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); + } + + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<CompletedStatistics, CompletedStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public CompletedStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers( + CompletedStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new CompletedStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.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 74% 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 index 9d7cf179ab1c..76c59cd5f4b8 100644 --- 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; @@ -29,29 +31,18 @@ * (sketching) can be used. */ @Internal -interface DataStatistics<D extends DataStatistics<D, S>, S> { +interface DataStatistics { + + StatisticsType type(); - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ boolean isEmpty(); /** Add row sortKey to data statistics. */ void add(SortKey sortKey); /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics + * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link + * ReservoirItemsSketch} (high cardinality) */ - S statistics(); + Object result(); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java similarity index 58% 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 c8ac79c61bf6..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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -35,60 +36,99 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.ThrowableCatchingRunnable; import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Comparators; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. + * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} + * every subtask and then merge them together. Once aggregation for all subtasks data statistics + * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link + * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the + * aggregated data statistics to improve data clustering. */ @Internal -class DataStatisticsCoordinator<D extends DataStatistics<D, S>, S> implements OperatorCoordinator { +class DataStatisticsCoordinator implements OperatorCoordinator { private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); private final String operatorName; + private final OperatorCoordinator.Context context; + private final Schema schema; + private final SortOrder sortOrder; + private final Comparator<StructLike> comparator; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; + private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; - private final transient AggregatedStatisticsTracker<D, S> aggregatedStatisticsTracker; - private volatile AggregatedStatistics<D, S> completedStatistics; - private volatile boolean started; + private final TypeSerializer<CompletedStatistics> completedStatisticsSerializer; + private final TypeSerializer<GlobalStatistics> globalStatisticsSerializer; + + private transient boolean started; + private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; DataStatisticsCoordinator( String operatorName, OperatorCoordinator.Context context, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { this.operatorName = operatorName; + this.context = context; + this.schema = schema; + this.sortOrder = sortOrder; + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; + this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker( + operatorName, + context.currentParallelism(), + schema, + sortOrder, + downstreamParallelism, + statisticsType, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + completedStatistics); } @Override public void close() throws Exception { coordinatorExecutor.shutdown(); + this.aggregatedStatisticsTracker = null; + this.started = false; LOG.info("Closed data statistics coordinator: {}.", operatorName); } @@ -148,7 +188,7 @@ private void runInCoordinatorThread(ThrowingRunnable<Throwable> action, String a operatorName, actionString, t); - operatorCoordinatorContext.failJob(t); + context.failJob(t); } }); } @@ -157,42 +197,108 @@ private void ensureStarted() { Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); } - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent<D, S> event) { - AggregatedStatistics<D, S> aggregatedStatistics = + private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { + CompletedStatistics maybeCompletedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + if (maybeCompletedStatistics != null) { + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator<StructLike> comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics<D, S> globalDataStatistics) { - callInCoordinatorThread( + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( () -> { - DataStatisticsEvent<D, S> dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); + for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference + subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } - - return null; }, String.format( "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } } @Override - @SuppressWarnings("unchecked") public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { runInCoordinatorThread( () -> { @@ -202,8 +308,14 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent<D, S>) event)); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -218,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); + 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)); } @@ -229,12 +346,10 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> r public void notifyCheckpointComplete(long checkpointId) {} @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, @@ -244,8 +359,13 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + this.completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); } @Override @@ -269,7 +389,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", subtask, attemptNumber, operatorName); @@ -295,14 +415,20 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics<D, S> completedStatistics() { + CompletedStatistics completedStatistics() { return completedStatistics; } + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + private static class SubtaskGateways { private final String operatorName; private final Map<Integer, SubtaskGateway>[] gateways; + @SuppressWarnings("unchecked") private SubtaskGateways(String operatorName, int parallelism) { this.operatorName = operatorName; gateways = new Map[parallelism]; diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java similarity index 62% 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 index 47dbfc3cfbe1..9d7d989c298e 100644 --- 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 @@ -19,33 +19,52 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; /** * DataStatisticsCoordinatorProvider provides the method to create new {@link * DataStatisticsCoordinator} */ @Internal -public class DataStatisticsCoordinatorProvider<D extends DataStatistics<D, S>, S> - extends RecreateOnResetOperatorCoordinator.Provider { +public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { private final String operatorName; - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; public DataStatisticsCoordinatorProvider( String operatorName, OperatorID operatorID, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { super(operatorID); this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + return new DataStatisticsCoordinator( + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); } } diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..7995a8a5b181 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +@Internal +public class DataStatisticsOperator extends AbstractStreamOperator<StatisticsOrRecord> + implements OneInputStreamOperator<RowData, StatisticsOrRecord>, OperatorEventHandler { + + private static final long serialVersionUID = 1L; + + private final String operatorName; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final OperatorEventGateway operatorEventGateway; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final TypeSerializer<DataStatistics> taskStatisticsSerializer; + private final TypeSerializer<GlobalStatistics> globalStatisticsSerializer; + + private transient int parallelism; + private transient int subtaskIndex; + private transient ListState<GlobalStatistics> globalStatisticsState; + // current statistics type may be different from the config due to possible + // migration from Map statistics to Sketch statistics when high cardinality detected + private transient volatile StatisticsType taskStatisticsType; + private transient volatile DataStatistics localStatistics; + private transient volatile GlobalStatistics globalStatistics; + + DataStatisticsOperator( + String operatorName, + Schema schema, + SortOrder sortOrder, + OperatorEventGateway operatorEventGateway, + int downstreamParallelism, + StatisticsType statisticsType) { + this.operatorName = operatorName; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.operatorEventGateway = operatorEventGateway; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + this.parallelism = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); + this.subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + + // Use union state so that new subtasks can also restore global statistics during scale-up. + this.globalStatisticsState = + context + .getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); + + if (context.isRestored()) { + if (globalStatisticsState.get() == null + || !globalStatisticsState.get().iterator().hasNext()) { + LOG.info( + "Operator {} subtask {} doesn't have global statistics state to restore", + operatorName, + subtaskIndex); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. + } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); + LOG.info( + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; + } + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); + } + + this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); + this.localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @Override + public void open() throws Exception { + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } + } + + @Override + public void handleOperatorEvent(OperatorEvent event) { + Preconditions.checkArgument( + event instanceof StatisticsEvent, + String.format( + "Operator %s subtask %s received unexpected operator event %s", + operatorName, subtaskIndex, event.getClass())); + StatisticsEvent statisticsEvent = (StatisticsEvent) event; + LOG.info( + "Operator {} subtask {} received global data event from coordinator checkpoint {}", + operatorName, + subtaskIndex, + statisticsEvent.checkpointId()); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); + checkStatisticsTypeMigration(); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } + } + + @Override + public void processElement(StreamRecord<RowData> streamRecord) { + // collect data statistics + RowData record = streamRecord.getValue(); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); + + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + long checkpointId = context.getCheckpointId(); + LOG.info( + "Operator {} subtask {} snapshotting data statistics for checkpoint {}", + operatorName, + subtaskIndex, + checkpointId); + + // Pass global statistics to partitioner so that all the operators refresh statistics + // at same checkpoint barrier + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } + + // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores + // an exact copy of globalStatistics + if (globalStatistics != null + && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { + globalStatisticsState.clear(); + LOG.info( + "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); + globalStatisticsState.add(globalStatistics); + LOG.debug( + "Operator {} subtask {} saved global statistics to state: {}", + operatorName, + subtaskIndex, + globalStatistics); + } + + // For now, local statistics are sent to coordinator at checkpoint + LOG.info( + "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", + operatorName, + subtaskIndex, + checkpointId); + operatorEventGateway.sendEventToCoordinator( + StatisticsEvent.createTaskStatisticsEvent( + checkpointId, localStatistics, taskStatisticsSerializer)); + + // Recreate the local statistics + localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @SuppressWarnings("unchecked") + private void checkStatisticsTypeMigration() { + // only check if the statisticsType config is Auto and localStatistics is currently Map type + if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { + Map<SortKey, Long> mapStatistics = (Map<SortKey, Long>) localStatistics.result(); + // convert if local statistics has cardinality over the threshold or + // if received global statistics is already sketch type + if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { + LOG.info( + "Operator {} subtask {} switched local statistics from Map to Sketch.", + operatorName, + subtaskIndex); + this.taskStatisticsType = StatisticsType.Sketch; + this.localStatistics = + StatisticsUtil.createTaskStatistics( + taskStatisticsType, parallelism, downstreamParallelism); + SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); + } + } + } + + @VisibleForTesting + DataStatistics localStatistics() { + return localStatistics; + } + + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory<StatisticsOrRecord> + implements CoordinatedOperatorFactory<StatisticsOrRecord>, + OneInputStreamOperatorFactory<RowData, StatisticsOrRecord> { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public <T extends StreamOperator<StatisticsOrRecord>> T createStreamOperator( + StreamOperatorParameters<StatisticsOrRecord> parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class<? extends StreamOperator> getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java new file mode 100644 index 000000000000..8ce99073836d --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class DataStatisticsSerializer extends TypeSerializer<DataStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final MapSerializer<SortKey, Long> mapSerializer; + private final SortKeySketchSerializer sketchSerializer; + + DataStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer<DataStatistics> duplicate() { + TypeSerializer<SortKey> duplicateSortKeySerializer = sortKeySerializer.duplicate(); + return (duplicateSortKeySerializer == sortKeySerializer) + ? this + : new DataStatisticsSerializer(duplicateSortKeySerializer); + } + + @Override + public DataStatistics createInstance() { + return new MapDataStatistics(); + } + + @SuppressWarnings("unchecked") + @Override + public DataStatistics copy(DataStatistics obj) { + StatisticsType statisticsType = obj.type(); + if (statisticsType == StatisticsType.Map) { + MapDataStatistics from = (MapDataStatistics) obj; + Map<SortKey, Long> fromStats = (Map<SortKey, Long>) from.result(); + Map<SortKey, Long> toStats = Maps.newHashMap(fromStats); + return new MapDataStatistics(toStats); + } else if (statisticsType == StatisticsType.Sketch) { + // because ReservoirItemsSketch doesn't expose enough public methods for cloning, + // this implementation adopted the less efficient serialization and deserialization. + SketchDataStatistics from = (SketchDataStatistics) obj; + ReservoirItemsSketch<SortKey> fromStats = (ReservoirItemsSketch<SortKey>) from.result(); + byte[] bytes = fromStats.toByteArray(sketchSerializer); + Memory memory = Memory.wrap(bytes); + ReservoirItemsSketch<SortKey> toStats = + ReservoirItemsSketch.heapify(memory, sketchSerializer); + return new SketchDataStatistics(toStats); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics copy(DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @SuppressWarnings("unchecked") + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + StatisticsType statisticsType = obj.type(); + statisticsTypeSerializer.serialize(obj.type(), target); + if (statisticsType == StatisticsType.Map) { + Map<SortKey, Long> mapStatistics = (Map<SortKey, Long>) obj.result(); + mapSerializer.serialize(mapStatistics, target); + } else if (statisticsType == StatisticsType.Sketch) { + ReservoirItemsSketch<SortKey> sketch = (ReservoirItemsSketch<SortKey>) obj.result(); + byte[] sketchBytes = sketch.toByteArray(sketchSerializer); + target.writeInt(sketchBytes.length); + target.write(sketchBytes); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataInputView source) throws IOException { + StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); + if (statisticsType == StatisticsType.Map) { + Map<SortKey, Long> mapStatistics = mapSerializer.deserialize(source); + return new MapDataStatistics(mapStatistics); + } else if (statisticsType == StatisticsType.Sketch) { + int numBytes = source.readInt(); + byte[] sketchBytes = new byte[numBytes]; + source.read(sketchBytes); + Memory sketchMemory = Memory.wrap(sketchBytes); + ReservoirItemsSketch<SortKey> sketch = + ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); + return new SketchDataStatistics(sketch); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsSerializer)) { + return false; + } + + DataStatisticsSerializer other = (DataStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<DataStatistics> snapshotConfiguration() { + return new DataStatisticsSerializerSnapshot(this); + } + + public static class DataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<DataStatistics, DataStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DataStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new DataStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..a7fe2b30b865 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer<GlobalStatistics> { + private final TypeSerializer<SortKey> sortKeySerializer; + private final EnumSerializer<StatisticsType> statisticsTypeSerializer; + private final ListSerializer<SortKey> rangeBoundsSerializer; + private final ListSerializer<Integer> intsSerializer; + private final ListSerializer<Long> longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer<SortKey> sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<GlobalStatistics> duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry<SortKey, KeyAssignment> entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map<SortKey, KeyAssignment> keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List<Integer> assignedSubtasks = intsSerializer.deserialize(source); + List<Long> subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List<SortKey> sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot<GlobalStatistics> snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot<GlobalStatistics, GlobalStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer<?>[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer<?>[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer<?>[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..781bcc646023 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List<Integer> assignedSubtasks; + private final List<Long> subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List<Integer> assignedSubtasks, + List<Long> subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List<Integer> assignedSubtasks() { + return assignedSubtasks; + } + + List<Long> subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** + * Select a subtask for the key. + * + * @return subtask id + */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..9d8167460a1b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map<SortKey, KeyAssignment> keyAssignments; + + MapAssignment(int numPartitions, Map<SortKey, KeyAssignment> keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map<SortKey, Long> mapStatistics, + double closeFileCostWeightPercentage, + Comparator<StructLike> comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map<SortKey, KeyAssignment> keyAssignments() { + return keyAssignments; + } + + /** + * Returns assignment summary for every subtask. + * + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map<Integer, Pair<Long, Integer>> assignmentInfo() { + Map<Integer, Pair<Long, Integer>> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair<Long, Integer> oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map<SortKey, KeyAssignment> assignment( + int numPartitions, + Map<SortKey, Long> mapStatistics, + double closeFileCostWeightPercentage, + Comparator<StructLike> comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap<SortKey, Long> sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map<SortKey, KeyAssignment> buildAssignment( + int numPartitions, + NavigableMap<SortKey, Long> sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map<SortKey, KeyAssignment> assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator<SortKey> mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List<Integer> assignedSubtasks = Lists.newArrayList(); + List<Long> subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java similarity index 58% 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 index 0b63e2721178..05b943f6046f 100644 --- 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 @@ -19,52 +19,70 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics<MapDataStatistics, Map<SortKey, Long>> { - private final Map<SortKey, Long> statistics; +class MapDataStatistics implements DataStatistics { + private final Map<SortKey, Long> keyFrequency; MapDataStatistics() { - this.statistics = Maps.newHashMap(); + this.keyFrequency = Maps.newHashMap(); } - MapDataStatistics(Map<SortKey, Long> statistics) { - this.statistics = statistics; + MapDataStatistics(Map<SortKey, Long> keyFrequency) { + this.keyFrequency = keyFrequency; + } + + @Override + public StatisticsType type() { + return StatisticsType.Map; } @Override public boolean isEmpty() { - return statistics.isEmpty(); + return keyFrequency.isEmpty(); } @Override public void add(SortKey sortKey) { - if (statistics.containsKey(sortKey)) { - statistics.merge(sortKey, 1L, Long::sum); + if (keyFrequency.containsKey(sortKey)) { + keyFrequency.merge(sortKey, 1L, Long::sum); } else { // clone the sort key before adding to map because input sortKey object can be reused SortKey copiedKey = sortKey.copy(); - statistics.put(copiedKey, 1L); + keyFrequency.put(copiedKey, 1L); } } @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + public Object result() { + return keyFrequency; } @Override - public Map<SortKey, Long> statistics() { - return statistics; + public String toString() { + return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); } @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof MapDataStatistics)) { + return false; + } + + MapDataStatistics other = (MapDataStatistics) o; + return Objects.equal(keyFrequency, other.keyFrequency); + } + + @Override + public int hashCode() { + return Objects.hashCode(keyFrequency); } } diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..f36a078c94e0 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + * <p>It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + * <p>All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner<RowData> { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final MapAssignment mapAssignment; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.mapAssignment = mapAssignment; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info( + "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", + newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + newSortKeyCounter = 0; + } + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..6608b938f5a8 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ +@Internal +public class RangePartitioner implements Partitioner<StatisticsOrRecord> { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner<RowData> delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner<RowData> delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..35bbb27baf16 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** MapDataStatistics uses map to count key frequency */ +class SketchDataStatistics implements DataStatistics { + + private final ReservoirItemsSketch<SortKey> sketch; + + SketchDataStatistics(int reservoirSize) { + this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); + } + + SketchDataStatistics(ReservoirItemsSketch<SortKey> sketchStats) { + this.sketch = sketchStats; + } + + @Override + public StatisticsType type() { + return StatisticsType.Sketch; + } + + @Override + public boolean isEmpty() { + return sketch.getNumSamples() == 0; + } + + @Override + public void add(SortKey sortKey) { + // clone the sort key first because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + sketch.update(copiedKey); + } + + @Override + public Object result() { + return sketch; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SketchDataStatistics)) { + return false; + } + + ReservoirItemsSketch<SortKey> otherSketch = ((SketchDataStatistics) o).sketch; + return Objects.equal(sketch.getK(), otherSketch.getK()) + && Objects.equal(sketch.getN(), otherSketch.getN()) + && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..dddb0d8722c0 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; + +class SketchRangePartitioner implements Partitioner<RowData> { + private final SortKey sortKey; + private final Comparator<StructLike> comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..871ef9ef1149 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; + +class SketchUtil { + static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; + static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; + static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; + static final int OPERATOR_OVER_SAMPLE_RATIO = 10; + + // switch the statistics tracking from map to sketch if the cardinality of the sort key is over + // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. + static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; + static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; + + private SketchUtil() {} + + /** + * The larger the reservoir size, the more accurate for range bounds calculation and the more + * balanced range distribution. + * + * <p>Here are the heuristic rules + * <li>Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the + * range bounds + * <li>Min is 10K to achieve good accuracy while memory footprint is still relatively small + * <li>Max is 1M to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @return reservoir size + */ + static int determineCoordinatorReservoirSize(int numPartitions) { + int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; + + if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { + // adjust it up and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); + } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { + // adjust it down and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; + } + + return reservoirSize; + } + + /** + * Determine the sampling reservoir size where operator subtasks collect data statistics. + * + * <p>Here are the heuristic rules + * <li>Target size is "coordinator reservoir size * over sampling ration (10) / operator + * parallelism" + * <li>Min is 1K to achieve good accuracy while memory footprint is still relatively small + * <li>Max is 100K to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @param operatorParallelism data statistics operator parallelism + * @return reservoir size + */ + static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { + int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); + int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; + return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); + } + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + * <ul> + * <li>age <= 15 + * <li>age > 15 && age <= 32 + * <li>age >32 && age <= 60 + * <li>age > 60 + * </ul> + * + * <p>Assumption is that a single key is not dominant enough to span multiple subtasks. + * + * @param numPartitions number of partitions which maps to downstream operator parallelism + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + */ + static SortKey[] rangeBounds( + int numPartitions, Comparator<StructLike> comparator, SortKey[] samples) { + // sort the keys first + Arrays.sort(samples, comparator); + int numCandidates = numPartitions - 1; + SortKey[] candidates = new SortKey[numCandidates]; + int step = (int) Math.ceil((double) samples.length / numPartitions); + int position = step - 1; + int numChosen = 0; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; + // skip duplicate values + if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { + // linear probe for the next distinct value + position += 1; + } else { + candidates[numChosen] = candidate; + position += step; + numChosen += 1; + } + } + + return candidates; + } + + /** This can be a bit expensive since it is quadratic. */ + static void convertMapToSketch( + Map<SortKey, Long> taskMapStats, Consumer<SortKey> sketchConsumer) { + taskMapStats.forEach( + (sortKey, count) -> { + for (int i = 0; i < count; ++i) { + sketchConsumer.accept(sortKey); + } + }); + } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator<StructLike> 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 91% 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 d03409f2a430..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 @@ -24,8 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; -import org.apache.flink.annotation.VisibleForTesting; +import java.util.stream.Collectors; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -41,6 +42,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer<SortKey> { @@ -276,13 +278,12 @@ public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot< private Schema schema; private SortOrder sortOrder; - @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { // this constructor is used when restoring from a checkpoint. } - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". @SuppressWarnings("checkstyle:RedundantModifier") public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { this.schema = schema; @@ -315,13 +316,30 @@ public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCode @Override public TypeSerializerSchemaCompatibility<SortKey> resolveSchemaCompatibility( - TypeSerializer<SortKey> newSerializer) { - if (!(newSerializer instanceof SortKeySerializer)) { + TypeSerializerSnapshot<SortKey> oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof SortKeySerializerSnapshot)) { return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; - return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + // Sort order should be identical + SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; + if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + Set<Integer> sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema oldSortSchema = TypeUtil.project(oldSnapshot.schema, sortFieldIds); + + List<String> compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, oldSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override @@ -337,17 +355,5 @@ private void readV1(DataInputView in) throws IOException { this.schema = SchemaParser.fromJson(schemaJson); this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); } - - @VisibleForTesting - static <T> TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List<String> compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } } } diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..d6c23f035015 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link + * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The + * implementation is modeled after {@link ArrayOfStringsSerDe} + */ +class SortKeySketchSerializer extends ArrayOfItemsSerDe<SortKey> implements Serializable { + private static final int DEFAULT_SORT_KEY_SIZE = 128; + + private final TypeSerializer<SortKey> itemSerializer; + private final ListSerializer<SortKey> listSerializer; + private final DataInputDeserializer input; + + SortKeySketchSerializer(TypeSerializer<SortKey> itemSerializer) { + this.itemSerializer = itemSerializer; + this.listSerializer = new ListSerializer<>(itemSerializer); + this.input = new DataInputDeserializer(); + } + + @Override + public byte[] serializeToByteArray(SortKey item) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); + itemSerializer.serialize(item, output); + byte[] itemBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[numBytes + Integer.BYTES]; + ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); + ByteArrayUtil.putIntLE(out, 0, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public byte[] serializeToByteArray(SortKey[] items) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); + listSerializer.serialize(Arrays.asList(items), output); + byte[] itemsBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[Integer.BYTES + numBytes]; + ByteArrayUtil.putIntLE(out, 0, numBytes); + System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return new SortKey[0]; + } + + long offset = startingOffset; + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + offset += Integer.BYTES; + + Util.checkBounds(offset, numBytes, mem.getCapacity()); + byte[] sortKeyBytes = new byte[numBytes]; + mem.getByteArray(offset, sortKeyBytes, 0, numBytes); + input.setBuffer(sortKeyBytes); + + try { + List<SortKey> sortKeys = listSerializer.deserialize(input); + SortKey[] array = new SortKey[numItems]; + sortKeys.toArray(array); + input.releaseArrays(); + return array; + } catch (IOException e) { + throw new UncheckedIOException("Failed to deserialize sort key sketch", e); + } + } + + @Override + public int sizeOf(SortKey item) { + return serializeToByteArray(item).length; + } + + @Override + public int sizeOf(Memory mem, long offset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return 0; + } + + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + return Integer.BYTES + numBytes; + } + + @Override + public String toString(SortKey item) { + return item.toString(); + } + + @Override + public Class<SortKey> getClassOfT() { + return SortKey.class; + } +} 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<SortField> sortFields = sortOrder.fields(); + int size = sortFields.size(); + List<Types.NestedField> 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/DataStatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 58% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index 852d2157b8cb..f6fcdb8b16ef 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -27,24 +27,39 @@ * statistics in bytes */ @Internal -class DataStatisticsEvent<D extends DataStatistics<D, S>, S> implements OperatorEvent { +class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; + private final boolean applyImmediately; - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; } - static <D extends DataStatistics<D, S>, S> DataStatisticsEvent<D, S> create( + static StatisticsEvent createTaskStatisticsEvent( long checkpointId, - DataStatistics<D, S> dataStatistics, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - return new DataStatisticsEvent<>( + DataStatistics statistics, + TypeSerializer<DataStatistics> statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. + return new StatisticsEvent( checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); + } + + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer<GlobalStatistics> statisticsSerializer, + boolean applyImmediately) { + return new StatisticsEvent( + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); } long checkpointId() { @@ -54,4 +69,8 @@ long checkpointId() { byte[] statisticsBytes() { return statisticsBytes; } + + boolean applyImmediately() { + return applyImmediately; + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 66% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index 889e85112e16..bc28df2b0e22 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.io.Serializable; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.data.RowData; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -34,68 +35,66 @@ * After shuffling, a filter and mapper are required to filter out the data distribution weight, * unwrap the object and extract the original record type T. */ -class DataStatisticsOrRecord<D extends DataStatistics<D, S>, S> implements Serializable { +@Internal +public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private DataStatistics<D, S> statistics; + private GlobalStatistics statistics; private RowData record; - private DataStatisticsOrRecord(DataStatistics<D, S> statistics, RowData record) { + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; this.record = record; } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); + static StatisticsOrRecord fromRecord(RowData record) { + return new StatisticsOrRecord(null, record); } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> fromDataStatistics( - DataStatistics<D, S> statistics) { - return new DataStatisticsOrRecord<>(statistics, null); + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { + return new StatisticsOrRecord(statistics, null); } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> reuseRecord( - DataStatisticsOrRecord<D, S> reuse, TypeSerializer<RowData> recordSerializer) { + static StatisticsOrRecord reuseRecord( + StatisticsOrRecord reuse, TypeSerializer<RowData> recordSerializer) { if (reuse.hasRecord()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } } - static <D extends DataStatistics<D, S>, S> DataStatisticsOrRecord<D, S> reuseStatistics( - DataStatisticsOrRecord<D, S> reuse, - TypeSerializer<DataStatistics<D, S>> statisticsSerializer) { - if (reuse.hasDataStatistics()) { + static StatisticsOrRecord reuseStatistics( + StatisticsOrRecord reuse, TypeSerializer<GlobalStatistics> statisticsSerializer) { + if (reuse.hasStatistics()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); } } - boolean hasDataStatistics() { + boolean hasStatistics() { return statistics != null; } - boolean hasRecord() { + public boolean hasRecord() { return record != null; } - DataStatistics<D, S> dataStatistics() { + GlobalStatistics statistics() { return statistics; } - void dataStatistics(DataStatistics<D, S> newStatistics) { + void statistics(GlobalStatistics newStatistics) { this.statistics = newStatistics; } - RowData record() { + public RowData record() { return record; } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 53% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index e9a6fa0cbfc5..d4ae2b359679 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -29,13 +29,12 @@ import org.apache.flink.table.data.RowData; @Internal -class DataStatisticsOrRecordSerializer<D extends DataStatistics<D, S>, S> - extends TypeSerializer<DataStatisticsOrRecord<D, S>> { - private final TypeSerializer<DataStatistics<D, S>> statisticsSerializer; +class StatisticsOrRecordSerializer extends TypeSerializer<StatisticsOrRecord> { + private final TypeSerializer<GlobalStatistics> statisticsSerializer; private final TypeSerializer<RowData> recordSerializer; - DataStatisticsOrRecordSerializer( - TypeSerializer<DataStatistics<D, S>> statisticsSerializer, + StatisticsOrRecordSerializer( + TypeSerializer<GlobalStatistics> statisticsSerializer, TypeSerializer<RowData> recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,13 +47,13 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() { - TypeSerializer<DataStatistics<D, S>> duplicateStatisticsSerializer = + public TypeSerializer<StatisticsOrRecord> duplicate() { + TypeSerializer<GlobalStatistics> duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer<RowData> duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( + return new StatisticsOrRecordSerializer( duplicateStatisticsSerializer, duplicateRowDataSerializer); } else { return this; @@ -62,34 +61,31 @@ public TypeSerializer<DataStatisticsOrRecord<D, S>> duplicate() { } @Override - public DataStatisticsOrRecord<D, S> createInstance() { + public StatisticsOrRecord createInstance() { // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } @Override - public DataStatisticsOrRecord<D, S> copy(DataStatisticsOrRecord<D, S> from) { + public StatisticsOrRecord copy(StatisticsOrRecord from) { if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); } } @Override - public DataStatisticsOrRecord<D, S> copy( - DataStatisticsOrRecord<D, S> from, DataStatisticsOrRecord<D, S> reuse) { - DataStatisticsOrRecord<D, S> to; + public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { + StatisticsOrRecord to; if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.copy(from.record(), to.record()); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics<D, S> statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); + to.statistics(statistics); } return to; @@ -101,41 +97,40 @@ public int getLength() { } @Override - public void serialize(DataStatisticsOrRecord<D, S> statisticsOrRecord, DataOutputView target) + public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) throws IOException { if (statisticsOrRecord.hasRecord()) { target.writeBoolean(true); recordSerializer.serialize(statisticsOrRecord.record(), target); } else { target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); } } @Override - public DataStatisticsOrRecord<D, S> deserialize(DataInputView source) throws IOException { + public StatisticsOrRecord deserialize(DataInputView source) throws IOException { boolean isRecord = source.readBoolean(); if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); } } @Override - public DataStatisticsOrRecord<D, S> deserialize( - DataStatisticsOrRecord<D, S> reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord<D, S> to; + public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) + throws IOException { + StatisticsOrRecord to; boolean isRecord = source.readBoolean(); if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.deserialize(to.record(), source); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics<D, S> statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + to.statistics(statistics); } return to; @@ -154,12 +149,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + if (!(obj instanceof StatisticsOrRecordSerializer)) { return false; } - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer<D, S> other = (DataStatisticsOrRecordSerializer<D, S>) obj; + StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; return Objects.equals(statisticsSerializer, other.statisticsSerializer) && Objects.equals(recordSerializer, other.recordSerializer); } @@ -170,25 +164,20 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot<DataStatisticsOrRecord<D, S>> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); + public TypeSerializerSnapshot<StatisticsOrRecord> snapshotConfiguration() { + return new StatisticsOrRecordSerializerSnapshot(this); } - public static class DataStatisticsOrRecordSerializerSnapshot<D extends DataStatistics<D, S>, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord<D, S>, DataStatisticsOrRecordSerializer<D, S>> { + public static class StatisticsOrRecordSerializerSnapshot + extends CompositeTypeSerializerSnapshot<StatisticsOrRecord, StatisticsOrRecordSerializer> { private static final int CURRENT_VERSION = 1; - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); - } + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public StatisticsOrRecordSerializerSnapshot() {} @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer<D, S> serializer) { + public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { super(serializer); } @@ -200,7 +189,7 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer<?>[] getNestedSerializers( - DataStatisticsOrRecordSerializer<D, S> outerSerializer) { + StatisticsOrRecordSerializer outerSerializer) { return new TypeSerializer<?>[] { outerSerializer.statisticsSerializer, outerSerializer.recordSerializer }; @@ -208,12 +197,12 @@ protected TypeSerializer<?>[] getNestedSerializers( @SuppressWarnings("unchecked") @Override - protected DataStatisticsOrRecordSerializer<D, S> createOuterSerializerWithNestedSerializers( + protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer<?>[] nestedSerializers) { - TypeSerializer<DataStatistics<D, S>> statisticsSerializer = - (TypeSerializer<DataStatistics<D, S>>) nestedSerializers[0]; + TypeSerializer<GlobalStatistics> statisticsSerializer = + (TypeSerializer<GlobalStatistics>) nestedSerializers[0]; TypeSerializer<RowData> recordSerializer = (TypeSerializer<RowData>) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } } } diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..43f72e336e06 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +/** + * Range distribution requires gathering statistics on the sort keys to determine proper range + * boundaries to distribute/cluster rows before writer operators. + */ +public enum StatisticsType { + /** + * Tracks the data statistics as {@code Map<SortKey, Long>} frequency. It works better for + * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in + * hundreds or thousands. + * + * <ul> + * <li>Pro: accurate measurement on the statistics/weight of every key. + * <li>Con: memory footprint can be large if the key cardinality is high. + * </ul> + */ + Map, + + /** + * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds + * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, + * uuid etc.) where the cardinalities can be in millions or billions. + * + * <ul> + * <li>Pro: relatively low memory footprint for high-cardinality sort keys. + * <li>Con: non-precise approximation with potentially lower accuracy. + * </ul> + */ + Sketch, + + /** + * Initially use Map for statistics tracking. If key cardinality turns out to be high, + * automatically switch to sketch sampling. + */ + Auto +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java new file mode 100644 index 000000000000..5d48ec57ca49 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.io.UncheckedIOException; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class StatisticsUtil { + + private StatisticsUtil() {} + + static DataStatistics createTaskStatistics( + StatisticsType type, int operatorParallelism, int numPartitions) { + if (type == StatisticsType.Map) { + return new MapDataStatistics(); + } else { + return new SketchDataStatistics( + SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); + } + } + + static byte[] serializeDataStatistics( + DataStatistics dataStatistics, TypeSerializer<DataStatistics> statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize data statistics", e); + } + } + + static DataStatistics deserializeDataStatistics( + byte[] bytes, TypeSerializer<DataStatistics> statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize data statistics", e); + } + } + + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer<CompletedStatistics> statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(completedStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer<CompletedStatistics> statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer<GlobalStatistics> statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer<GlobalStatistics> statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static StatisticsType collectType(StatisticsType config) { + return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; + } + + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + + static StatisticsType collectType( + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.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 77% 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..5718f4b93825 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 @@ -28,6 +28,8 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +39,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -61,10 +66,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -72,6 +79,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; @@ -95,6 +103,11 @@ public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEn private final SerializableRecordEmitter<T> emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List<IcebergSourceSplit> batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -130,16 +143,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List<IcebergSourceSplit> planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List<IcebergSourceSplit> splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -205,26 +228,69 @@ private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumer // Only do scan planning if nothing is restored from checkpoint state List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static <T> Builder<T> builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder<RowData> forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param <T> output type + * @return an IcebergSource builder + */ + public static <T> Builder<T> forOutputType(RowDataConverter<T> converter) { + return new Builder<T>().converter(converter); + } + public static class Builder<T> { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator<IcebergSourceSplit> splitComparator; private ReaderFunction<T> readerFunction; + private RowDataConverter<T> converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +321,28 @@ public Builder<T> splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder<T> readerFunction(ReaderFunction<T> newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder<T> converter(RowDataConverter<T> newConverter) { + this.converter = newConverter; + return this; + } + public Builder<T> flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -482,6 +565,10 @@ public IcebergSource<T> build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); @@ -506,24 +593,7 @@ public IcebergSource<T> build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters()); - this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -544,5 +614,75 @@ public IcebergSource<T> build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream<T> buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource<T> source = build(); + TypeInformation<T> outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource<T> stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static <T> TypeInformation<T> outputTypeInfo( + RowDataConverter<T> converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation<T>) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + + private ReaderFunction<T> readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction<T>) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction<T>) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.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 89% 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 index 610657e8d47b..65adce77d9f9 100644 --- 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 @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource<RowData> createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream<RowData> createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource<RowData> source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.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<String> 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<IcebergSourceSplit, IcebergEnumeratorState> { + implements SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState>, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext<IcebergSourceSplit> 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<IcebergSourceSplit> 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<IcebergSourceSplit> 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.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter<GenericRecord> { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation<GenericRecord> outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation<GenericRecord> getProducedType() { + return outputTypeInfo; + } +} 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 92% 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 index 66e59633fff2..f89e5ce13474 100644 --- 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 @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction<GenericRecord> { private final String tableName; private final Schema readSchema; 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.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction<T> extends DataIteratorReaderFunction<T> { + private final RowDataConverter<T> converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List<Expression> filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter<T> converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List<Expression> filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator<T> createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader<T> implements FileScanTaskReader<T> { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter<T> converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter<T> converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator<T> open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/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 98% 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 index 9c20494fdbcd..bcd72e25036b 100644 --- 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 @@ -85,7 +85,7 @@ public RecordsWithSplitIds<RecordAndPosition<T>> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/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<T> extends DataIterator<T> { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader<T> 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.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param <T> output type + */ +public interface RowDataConverter<T> + extends Function<RowData, T>, ResultTypeQueryable<T>, Serializable {} 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<RowData> { private final FileIO io; private final EncryptionManager encryption; private final List<Expression> 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<Expression> 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<Expression> 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<RowData> 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<Long> { + 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<String, String> props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry<String, String> 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 79% 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 index 9a73b80e077d..d2e086aa448e 100644 --- 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 @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; @@ -50,4 +51,17 @@ public static MiniClusterExtension createWithClassloaderCheckDisabled() { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/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 98% 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 index ce6caca12158..1767f774922a 100644 --- 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Duration; @@ -71,7 +72,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; -import org.junit.Assert; public class SimpleDataUtil { @@ -268,13 +268,13 @@ public static boolean equalsRecords(List<Record> expected, List<Record> actual, } public static void assertRecordsEqual(List<Record> expected, List<Record> actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); + assertThat(actual).hasSameSizeAs(expected); Types.StructType type = schema.asStruct(); StructLikeSet expectedSet = StructLikeSet.create(type); expectedSet.addAll(expected); StructLikeSet actualSet = StructLikeSet.create(type); actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } /** @@ -297,7 +297,7 @@ public static void assertTableRecords(Table table, List<Record> expected, String Snapshot snapshot = latestSnapshot(table, branch); if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); + assertThat(expected).isEmpty(); return; } @@ -313,7 +313,7 @@ public static void assertTableRecords(Table table, List<Record> expected, String actualSet.add(record); } - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } } 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<Row> sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator<Row> iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable<Row> expected, Iterable<Row> actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable<Row> expected, Iterable<Row> 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<String, String> props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry<String, String> 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<Row> 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 95% 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 index 884ea2d1d3b1..b9a7d5b1d589 100644 --- 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 @@ -46,6 +46,8 @@ private TestFixtures() {} public static final String SINK_TABLE = "t_sink"; public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); public static final Schema TS_SCHEMA = new Schema( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/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<String, String> 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<Row> 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<String, String> 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<Row> 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<Long, List<DataFile>> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List<DataFile> 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..b63547d433a4 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List<Object[]> 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<String, String> 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<Row> 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<Row> 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<String, String> 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<Long, List<DataFile>> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List<DataFile> dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List<List<Row>> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List<Row> charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List<Row> flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map<String, String> tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.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<String, String> 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<Record> expected = expectedRecords.iterator(); Iterator<RowData> 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<RowData> expected = expectedRows.iterator(); Iterator<Record> 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 64% 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 df2e6ae21c7e..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 @@ -18,8 +18,12 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.withPrecision; + import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; @@ -37,19 +41,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestRowProjection { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { - File file = temp.newFile(desc + ".avro"); - Assert.assertTrue(file.delete()); + File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); + assertThat(file.delete()).isTrue(); try (FileAppender<RowData> appender = Avro.write(Files.localOutput(file)) @@ -79,10 +81,8 @@ public void testFullProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, schema, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data value", cmp, 0); + assertThat(projected.getLong(0)).isEqualTo(34); + assertThat(projected.getString(1)).asString().isEqualTo("test"); } @Test @@ -96,19 +96,13 @@ public void testSpecialCharacterProjection() throws Exception { RowData full = writeAndRead("special_chars", schema, schema, row); - Assert.assertEquals("Should contain the correct id value", 34L, full.getLong(0)); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", full.getString(1).toString())); + assertThat(full.getLong(0)).isEqualTo(34L); + assertThat(full.getString(1)).asString().isEqualTo("test"); RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); - Assert.assertEquals("Should not contain id value", 1, projected.getArity()); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", projected.getString(0).toString())); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getString(0)).asString().isEqualTo("test"); } @Test @@ -127,9 +121,8 @@ public void testReorderedFullProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, reordered, row); - Assert.assertEquals( - "Should contain the correct 0 value", "test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.getLong(1)); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + assertThat(projected.getLong(1)).isEqualTo(34); } @Test @@ -149,10 +142,9 @@ public void testReorderedProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, reordered, row); - Assert.assertTrue("Should contain the correct 0 value", projected.isNullAt(0)); - Assert.assertEquals( - "Should contain the correct 1 value", "test", projected.getString(1).toString()); - Assert.assertTrue("Should contain the correct 2 value", projected.isNullAt(2)); + assertThat(projected.isNullAt(0)).isTrue(); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + assertThat(projected.isNullAt(2)).isTrue(); } @Test @@ -173,10 +165,16 @@ public void testRenamedAddedField() throws Exception { Types.NestedField.optional(4, "d", Types.LongType.get())); RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertEquals("Should contain the correct value in column 2", projected.getLong(1), 200L); - Assert.assertEquals("Should contain the correct value in column 3", projected.getLong(2), 300L); - Assert.assertTrue("Should contain empty value on new column 4", projected.isNullAt(3)); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.getLong(1)) + .as("Should contain the correct value in column 2") + .isEqualTo(200L); + assertThat(projected.getLong(2)) + .as("Should contain the correct value in column 1") + .isEqualTo(300L); + assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } @Test @@ -190,8 +188,8 @@ public void testEmptyProjection() throws Exception { RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); - Assert.assertNotNull("Should read a non-null record", projected); - Assert.assertEquals(0, projected.getArity()); + assertThat(projected).isNotNull(); + assertThat(projected.getArity()).isEqualTo(0); } @Test @@ -206,16 +204,16 @@ public void testBasicProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); - Assert.assertEquals("Should not project data", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + assertThat(projected.getArity()).as("Should not project data").isEqualTo(1); + assertThat(projected.getLong(0)).isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(projected.getString(0)).asString().isEqualTo("test"); } @Test @@ -234,9 +232,11 @@ public void testRename() throws Exception { RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getString(1)) + .as("Should contain the correct data/renamed value") + .asString() + .isEqualTo("test"); } @Test @@ -257,8 +257,8 @@ public void testNestedStructProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should not project location", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getLong(0)).as("Should contain the correct id value").isEqualTo(34L); Schema latOnly = new Schema( @@ -269,11 +269,12 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); RowData projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project longitude", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project longitude").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); Schema longOnly = new Schema( @@ -284,21 +285,24 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project latitutde", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(0), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project latitutde").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(1), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedLocation.getFloat(1)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); } @Test @@ -324,23 +328,23 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project properties map", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project properties map").isEqualTo(1); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); } private Map<String, ?> toStringMap(Map<?, ?> map) { @@ -381,42 +385,50 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project locations map", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project locations map").isEqualTo(1); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project locations map", row.getMap(1), projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(row.getMap(1)); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); GenericMapData locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); + assertThat(locations).isNotNull(); GenericArrayData l1l2Array = new GenericArrayData( new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should contain lat", 53.992811f, projectedL1.getFloat(0), 0.000001); - Assert.assertEquals("L1 should not contain long", 1, projectedL1.getArity()); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain lat") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + assertThat(projectedL1.getArity()).as("L1 should not contain long").isEqualTo(1); RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should contain lat", 52.995143f, projectedL2.getFloat(0), 0.000001); - Assert.assertEquals("L2 should not contain long", 1, projectedL2.getArity()); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain lat") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedL2.getArity()).as("L2 should not contain long").isEqualTo(1); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should not contain lat", 1, projectedL1.getArity()); - Assert.assertEquals("L1 should contain long", -1.542616f, projectedL1.getFloat(0), 0.000001); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getArity()).as("L1 should not contain lat").isEqualTo(1); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain long") + .isEqualTo(-1.542616f, withPrecision(0.000001f)); projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should not contain lat", 1, projectedL2.getArity()); - Assert.assertEquals("L2 should contain long", -1.539054f, projectedL2.getFloat(0), 0.000001); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getArity()).as("L2 should not contain lat").isEqualTo(1); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain long") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -431,18 +443,20 @@ public void testMapOfStructsProjection() throws IOException { Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", 53.992811f, projectedL1.getFloat(0), 0.000001); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain latitude") + .isEqualTo(53.992811f, withPrecision(0.000001f)); projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", 52.995143f, projectedL2.getFloat(0), 0.000001); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); } @Test @@ -460,18 +474,18 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project values list", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project values list").isEqualTo(1); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); } @Test @@ -497,36 +511,36 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project points list", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).isEqualTo(1); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project points list", row.getArray(1), projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(row.getArray(1)); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); ArrayData points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); RowData projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should project x", 1, projectedP1.getInt(0)); - Assert.assertEquals("Should not project y", 1, projectedP1.getArity()); + assertThat(projectedP1.getInt(0)).as("Should project x").isEqualTo(1); + assertThat(projectedP1.getArity()).as("Should not project y").isEqualTo(1); RowData projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project y", 1, projectedP2.getArity()); - Assert.assertEquals("Should project x", 3, projectedP2.getInt(0)); + assertThat(projectedP2.getArity()).as("Should not project y").isEqualTo(1); + assertThat(projectedP2.getInt(0)).as("Should project x").isEqualTo(3); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x", 1, projectedP1.getArity()); - Assert.assertEquals("Should project y", 2, projectedP1.getInt(0)); + assertThat(projectedP1.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project y").isEqualTo(2); projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null y", projectedP2.isNullAt(0)); + assertThat(projectedP2.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null y").isTrue(); Schema yRenamed = new Schema( @@ -539,16 +553,16 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP1.getArity()); - Assert.assertEquals("Should project z", 2, projectedP1.getInt(0)); + assertThat(projectedP1.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project z").isEqualTo(2); projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null z", projectedP2.isNullAt(0)); + assertThat(projectedP2.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } @Test @@ -572,9 +586,11 @@ public void testAddedFieldsWithRequiredChildren() throws Exception { RowData projected = writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertTrue("Should contain empty value in new column 2", projected.isNullAt(1)); - Assert.assertTrue("Should contain empty value in new column 4", projected.isNullAt(2)); - Assert.assertTrue("Should contain empty value in new column 6", projected.isNullAt(3)); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.isNullAt(1)).as("Should contain empty value in new column 2").isTrue(); + assertThat(projected.isNullAt(2)).as("Should contain empty value in new column 4").isTrue(); + assertThat(projected.isNullAt(3)).as("Should contain empty value in new column 6").isTrue(); } } diff --git a/flink/v1.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 98% 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 index e0340e0743b0..eccab20e04fc 100644 --- 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 @@ -22,7 +22,7 @@ import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; import org.apache.iceberg.flink.TestHelpers; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStructRowData { diff --git a/flink/v1.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<T> implements Sink<T> { + private static final long serialVersionUID = 1L; + private static final List<BlockingQueue<Object>> 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<T> remainingOutput() { + return Lists.newArrayList((BlockingQueue<T>) QUEUES.get(this.index)); + } + + /** + * Check if there is no remaining output received by this {@link Sink}. + * + * @return <code>true</code> 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<T> createWriter(InitContext context) { + return new CollectingWriter<>(index); + } + + private static class CollectingWriter<T> implements SinkWriter<T> { + 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/ConstantsForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java new file mode 100644 index 000000000000..36e162d4f068 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +class ConstantsForTests { + public static final long EVENT_TIME = 10L; + static final long EVENT_TIME_2 = 11L; + static final String DUMMY_NAME = "dummy"; + + private ConstantsForTests() { + // Do not instantiate + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/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<String, String> catalogProperties; + private final String databaseName; + private final Path warehouse; + private final CatalogLoader catalogLoader; + private TableEnvironment tableEnvironment; + + public FlinkSqlExtension( + String catalogName, Map<String, String> 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<Row> 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<Path> 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<Row> exec(String query, Object... parameters) { + TableResult tableResult = tableEnvironment.executeSql(String.format(query, parameters)); + try (CloseableIterator<Row> 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<String, String> 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 <code>null</code> 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<T> + implements Source<T, ManualSource.DummySplit, ManualSource.DummyCheckpoint>, + ResultTypeQueryable<T> { + + private static final long serialVersionUID = 1L; + private static final List<ArrayDeque<Tuple2<?, Long>>> QUEUES = + Collections.synchronizedList(Lists.newArrayList()); + private static final List<CompletableFuture<Void>> AVAILABILITIES = + Collections.synchronizedList(Lists.newArrayList()); + private static int numSources = 0; + private final TypeInformation<T> type; + private final int index; + private transient DataStream<T> 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<T> 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<T> dataStream() { + if (this.stream == null) { + this.stream = + this.env + .fromSource(this, WatermarkStrategy.noWatermarks(), "ManualSource-" + index, type) + .forceNonParallel(); + } + + return this.stream; + } + + private void sendInternal(Tuple2<?, Long> tuple) { + QUEUES.get(index).offer(tuple); + AVAILABILITIES.get(index).complete(null); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SplitEnumerator<DummySplit, DummyCheckpoint> createEnumerator( + SplitEnumeratorContext<DummySplit> enumContext) { + return new DummyCheckpointEnumerator(); + } + + @Override + public SplitEnumerator<DummySplit, DummyCheckpoint> restoreEnumerator( + SplitEnumeratorContext<DummySplit> enumContext, DummyCheckpoint checkpoint) { + return new DummyCheckpointEnumerator(); + } + + @Override + public SimpleVersionedSerializer<DummySplit> getSplitSerializer() { + return new NoOpDummySplitSerializer(); + } + + @Override + public SimpleVersionedSerializer<DummyCheckpoint> getEnumeratorCheckpointSerializer() { + return new NoOpDummyCheckpointSerializer(); + } + + @Override + public SourceReader<T, DummySplit> createReader(SourceReaderContext sourceReaderContext) { + return new SourceReader<T, DummySplit>() { + @Override + public void start() { + // Do nothing + } + + @Override + public InputStatus pollNext(ReaderOutput<T> output) { + Tuple2<T, Long> next = (Tuple2<T, Long>) 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<DummySplit> snapshotState(long checkpointId) { + return Lists.newArrayList(new DummySplit()); + } + + @Override + public CompletableFuture<Void> isAvailable() { + return AVAILABILITIES.get(index); + } + + @Override + public void addSplits(List<DummySplit> splits) { + // do nothing + } + + @Override + public void notifyNoMoreSplits() { + // do nothing + } + + @Override + public void close() { + // do nothing + } + }; + } + + @Override + public TypeInformation<T> 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<DummySplit, DummyCheckpoint> { + + @Override + public void start() { + // do nothing + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // do nothing + } + + @Override + public void addSplitsBack(List<DummySplit> 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<DummySplit> { + @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<DummyCheckpoint> { + @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/MetricsReporterFactoryForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java new file mode 100644 index 000000000000..7a523035b7fb --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class MetricsReporterFactoryForTests implements MetricReporterFactory { + private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); + private static final Pattern FULL_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static Map<String, Counter> counters = Maps.newConcurrentMap(); + private static Map<String, Gauge> gauges = Maps.newConcurrentMap(); + private static Set<String> monitoredMetricNames; + + public MetricsReporterFactoryForTests() { + monitoredMetricNames = + Arrays.stream(TableMaintenanceMetrics.class.getDeclaredFields()) + .map( + f -> { + try { + return f.get(null).toString(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + } + + @Override + public MetricReporter createMetricReporter(Properties properties) { + return INSTANCE; + } + + public static void reset() { + counters = Maps.newConcurrentMap(); + gauges = Maps.newConcurrentMap(); + } + + public static Long counter(String name) { + return counterValues().get(name); + } + + public static Long gauge(String name) { + return gaugeValues().get(name); + } + + public static void assertGauges(Map<String, Long> expected) { + assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + } + + public static void assertCounters(Map<String, Long> expected) { + assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + } + + private static Map<String, Long> gaugeValues() { + return gauges.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> (Long) entry.getValue().getValue())); + } + + private static Map<String, Long> counterValues() { + return counters.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> entry.getValue().getCount())); + } + + private static Map<String, Long> filter(Map<String, Long> original, Map<String, Long> filter) { + return original.entrySet().stream() + .filter( + entry -> { + Long filterValue = filter.get(entry.getKey()); + return filterValue == null || filterValue != -1; + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static String longName(String fullName) { + Matcher matcher = FULL_METRIC_NAME.matcher(fullName); + if (!matcher.matches()) { + throw new RuntimeException(String.format("Can't parse simplified metrics name %s", fullName)); + } + + return matcher.group(1) + "." + matcher.group(2) + "." + matcher.group(3); + } + + private static class TestMetricsReporter implements MetricReporter { + @Override + public void open(MetricConfig config) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + if (monitoredMetricNames.contains(metricName)) { + if (metric instanceof Counter) { + counters.put(group.getMetricIdentifier(metricName), (Counter) metric); + } + + if (metric instanceof Gauge) { + gauges.put(group.getMetricIdentifier(metricName), (Gauge) metric); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + // do nothing + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java new file mode 100644 index 000000000000..225853086545 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; + +import java.io.File; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.extension.RegisterExtension; + +class OperatorTestBase { + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + private static final TriggerLockFactory.Lock MAINTENANCE_LOCK = new MemoryLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new MemoryLock(); + + static final String TABLE_NAME = "test_table"; + + @RegisterExtension + protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(config()) + .build()); + + @RegisterExtension + final FlinkSqlExtension sql = + new FlinkSqlExtension( + "catalog", + ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), + "db"); + + private static Configuration config() { + Configuration config = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + MetricOptions.forReporter(config, "test_reporter") + .set(MetricOptions.REPORTER_FACTORY_CLASS, MetricsReporterFactoryForTests.class.getName()); + return config; + } + + protected static TriggerLockFactory lockFactory() { + return new TriggerLockFactory() { + @Override + public void open() { + MAINTENANCE_LOCK.unlock(); + RECOVERY_LOCK.unlock(); + } + + @Override + public Lock createLock() { + return MAINTENANCE_LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // do nothing + } + }; + } + + /** + * Close the {@link JobClient} and wait for the job closure. If the savepointDir is specified, it + * stops the job with a savepoint. + * + * @param jobClient the job to close + * @param savepointDir the savepointDir to store the last savepoint. If <code>null</code> then + * stop without a savepoint. + * @return configuration for restarting the job from the savepoint + */ + public static Configuration closeJobClient(JobClient jobClient, File savepointDir) { + Configuration conf = new Configuration(); + if (jobClient != null) { + if (savepointDir != null) { + // Stop with savepoint + jobClient.stopWithSavepoint(false, savepointDir.getPath(), SavepointFormatType.CANONICAL); + // Wait until the savepoint is created and the job has been stopped + Awaitility.await().until(() -> savepointDir.listFiles(File::isDirectory).length == 1); + conf.set( + SavepointConfigOptions.SAVEPOINT_PATH, + savepointDir.listFiles(File::isDirectory)[0].getAbsolutePath()); + } else { + jobClient.cancel(); + } + + // Wait until the job has been stopped + Awaitility.await().until(() -> jobClient.getJobStatus().get().isTerminalState()); + return conf; + } + + return null; + } + + /** + * Close the {@link JobClient} and wait for the job closure. + * + * @param jobClient the job to close + */ + public static void closeJobClient(JobClient jobClient) { + closeJobClient(jobClient, null); + } + + private static class MemoryLock implements TriggerLockFactory.Lock { + boolean locked = false; + + @Override + public boolean tryLock() { + if (locked) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java new file mode 100644 index 000000000000..051d09d92bad --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +class TestJdbcLockFactory extends TestLockFactoryBase { + @Override + TriggerLockFactory lockFactory() { + return lockFactory("tableName"); + } + + @Test + void testMultiTableLock() { + JdbcLockFactory other = lockFactory("tableName2"); + other.open((JdbcLockFactory) this.lockFactory); + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = other.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + private JdbcLockFactory lockFactory(String tableName) { + Map<String, String> properties = Maps.newHashMap(); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(INIT_LOCK_TABLES_PROPERTY, "true"); + + return new JdbcLockFactory( + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), + tableName, + properties); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java new file mode 100644 index 000000000000..bf9e86f2534d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +abstract class TestLockFactoryBase { + protected TriggerLockFactory lockFactory; + + abstract TriggerLockFactory lockFactory(); + + @BeforeEach + void before() { + this.lockFactory = lockFactory(); + lockFactory.open(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testTryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock1.tryLock()).isFalse(); + assertThat(lock2.tryLock()).isFalse(); + } + + @Test + void testUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + } + + @Test + void testNoConflictWithRecoveryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createRecoveryLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + @Test + void testDoubleUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + assertThat(lock.tryLock()).isFalse(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java new file mode 100644 index 000000000000..3aee05322561 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -0,0 +1,371 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.assertj.core.api.Assertions.assertThat; +import 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.ContentFile; +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<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink<TableChange> 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<Record> 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( + TableChange.builder() + .dataFileCount(1) + .dataFileSizeInBytes(size) + .commitCount(1) + .build()); + }); + } 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<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink<TableChange> result = new CollectingSink<>(); + events.sinkTo(result); + + // Start the job + Configuration conf; + JobClient jobClient = null; + AtomicReference<TableChange> 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<TableChange> 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<TableChange> 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().commitCount()).isEqualTo(1); + + iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); + + // Expecting 2 commits/snapshots for maxReadBack=2 + assertThat(iterator.next().commitCount()).isEqualTo(2); + + iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + + // For maxReadBack Long.MAX_VALUE we get every change + assertThat(iterator.next().commitCount()).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().commitCount()).isEqualTo(1); + + // Create a DataOperations.REPLACE snapshot + Table table = tableLoader.loadTable(); + DataFile dataFile = + table.snapshots().iterator().next().addedDataFiles(table.io()).iterator().next(); + RewriteFiles rewrite = tableLoader.loadTable().newRewrite(); + // 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<DataFile> dataFiles = + Lists.newArrayList(table.currentSnapshot().addedDataFiles(table.io()).iterator()); + List<DeleteFile> deleteFiles = + Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); + + long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); + + TableChange newChange = previous.copy(); + newChange.merge( + TableChange.builder() + .dataFileCount(dataFiles.size()) + .dataFileSizeInBytes(dataSize) + // Currently we only test with equality deletes + .eqDeleteFileCount(deleteFiles.size()) + .eqDeleteRecordCount(deleteRecordCount) + .commitCount(1) + .build()); + return newChange; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java new file mode 100644 index 000000000000..fba4a12d9c6b --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -0,0 +1,686 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.DUMMY_NAME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME; +import static org.apache.iceberg.flink.maintenance.operator.ConstantsForTests.EVENT_TIME_2; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.GROUP_VALUE_DEFAULT; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.NOTHING_TO_TRIGGER; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED; +import static org.apache.iceberg.flink.maintenance.operator.TableMaintenanceMetrics.TRIGGERED; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +class TestTriggerManager extends OperatorTestBase { + private static final long DELAY = 10L; + private static final String NAME_1 = "name1"; + private static final String NAME_2 = "name2"; + private long processingTime = 0L; + private TriggerLockFactory lockFactory; + private TriggerLockFactory.Lock lock; + private TriggerLockFactory.Lock recoveringLock; + + @BeforeEach + void before() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + this.lockFactory = lockFactory(); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveringLock = lockFactory.createRecoveryLock(); + lock.unlock(); + recoveringLock.unlock(); + MetricsReporterFactoryForTests.reset(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testCommitCount() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 4); + } + } + + @Test + void testDataFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 0); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(5).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 4); + } + } + + @Test + void testDataFileSizeInBytes() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 3); + } + } + + @Test + void testPosDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 4); + } + } + + @Test + void testPosDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testEqDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 4); + } + } + + @Test + void testEqDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(5L).build(), 2); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 3); + } + } + + @Test + void testTimeout() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + TableChange event = TableChange.builder().dataFileCount(1).commitCount(1).build(); + + // Wait for some time + testHarness.processElement(event, EVENT_TIME); + assertThat(testHarness.extractOutputValues()).isEmpty(); + + // Wait for the timeout to expire + long newTime = EVENT_TIME + Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // Send a new event + testHarness.setProcessingTime(newTime + 1); + testHarness.processElement(event, newTime); + + // No trigger yet + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Send a new event + newTime += Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + + // New trigger should arrive + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testStateRestore() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + testHarness.processElement( + TableChange.builder().dataFileCount(1).commitCount(1).build(), EVENT_TIME); + + assertThat(testHarness.extractOutputValues()).isEmpty(); + + state = testHarness.snapshot(1, EVENT_TIME); + } + + // Restore the state, write some more data, create a checkpoint, check the data which is written + manager = manager(tableLoader); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + // Arrives the first real change which triggers the recovery process + testHarness.processElement(TableChange.builder().commitCount(1).build(), EVENT_TIME_2); + assertTriggers( + testHarness.extractOutputValues(), + Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); + + // Remove the lock to allow the next trigger + recoveringLock.unlock(); + testHarness.setProcessingTime(EVENT_TIME_2); + // At this point the output contains the recovery trigger and the real trigger + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testMinFireDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, DELAY, 1); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // No new fire yet + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testLockCheckDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, 1, DELAY); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + + // Create a lock to prevent execution, and check that there is no result + assertThat(lock.tryLock()).isTrue(); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // Remove the lock, and still no trigger + lock.unlock(); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + /** + * Simulating recovery scenarios where there is a leftover table lock, and ongoing maintenance + * task. + * + * @param locked if a lock exists on the table on job recovery + * @param runningTask is running and continues to run after job recovery + */ + @ParameterizedTest + @MethodSource("parametersForTestRecovery") + void testRecovery(boolean locked, boolean runningTask) throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.open(); + state = testHarness.snapshot(1, EVENT_TIME); + } + + if (locked) { + assertThat(lock.tryLock()).isTrue(); + } + + manager = manager(tableLoader); + List<Trigger> expected = Lists.newArrayListWithExpectedSize(3); + try (KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + ++processingTime; + expected.add(Trigger.recovery(processingTime)); + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Nothing happens until the recovery is finished + ++processingTime; + testHarness.setProcessingTime(processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + if (runningTask) { + // Simulate the action of the recovered maintenance task lock removal when it finishes + lock.unlock(); + } + + // Still no results as the recovery is ongoing + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it + // received recovery trigger + lock.unlock(); + recoveringLock.unlock(); + + // Emit only a single trigger + ++processingTime; + testHarness.setProcessingTime(processingTime); + // Releasing lock will create a new snapshot, and we receive this in the trigger + expected.add( + Trigger.create( + processingTime, + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()), + 0)); + assertTriggers(testHarness.extractOutputValues(), expected); + } + } + + @Test + void testTriggerMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TableChange> source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink<Trigger> sink = new CollectingSink<>(); + + TriggerManager manager = + new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1, NAME_2), + Lists.newArrayList( + new TriggerEvaluator.Builder().commitCount(2).build(), + new TriggerEvaluator.Builder().commitCount(4).build()), + 1L, + 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // This one doesn't trigger - tests NOTHING_TO_TRIGGER + source.sendRecord(TableChange.builder().commitCount(1).build()); + + Awaitility.await() + .until( + () -> { + Long notingCounter = + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + return notingCounter != null && notingCounter.equals(1L); + }); + + // Trigger one of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(1).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(1L); + lock.unlock(); + + // Trigger both of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitCount(2).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(2L); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED)) + .isEqualTo(1L); + + // Final check all the counters + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder<String, Long>() + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, -1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, -1L) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 2L) + .put(DUMMY_NAME + "." + NAME_2 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 1L) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testRateLimiterMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TableChange> source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink<Trigger> sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1_000_000L, 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // The second trigger will be blocked + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + .equals(1L)); + + // Final check all the counters + assertCounters(1L, 0L); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testConcurrentRunMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource<TableChange> source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink<Trigger> sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1L, 1_000_000L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger - notice that we do not remove the lock after the trigger + source.sendRecord(TableChange.builder().commitCount(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // The second trigger will be blocked by the lock + source.sendRecord(TableChange.builder().commitCount(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + .equals(1L)); + + // Final check all the counters + assertCounters(0L, 1L); + } finally { + closeJobClient(jobClient); + } + } + + private static Stream<Arguments> parametersForTestRecovery() { + return Stream.of( + Arguments.of(true, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false)); + } + + private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder<String, Long>() + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + rateLimiterTrigger) + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + concurrentRunTrigger) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .build()); + } + + private KeyedOneInputStreamOperatorTestHarness<Boolean, TableChange, Trigger> harness( + TriggerManager manager) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new KeyedProcessOperator<>(manager), value -> true, Types.BOOLEAN); + } + + private void addEventAndCheckResult( + OneInputStreamOperatorTestHarness<TableChange, Trigger> testHarness, + TableChange event, + int expectedSize) + throws Exception { + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(event, processingTime); + assertThat(testHarness.extractOutputValues()).hasSize(expectedSize); + // Remove the lock to allow the next trigger + lock.unlock(); + } + + private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { + return new TriggerManager( + tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + } + + private TriggerManager manager( + TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { + return new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1), + Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), + minFireDelayMs, + lockCheckDelayMs); + } + + private TriggerManager manager(TableLoader tableLoader) { + return manager(tableLoader, new TriggerEvaluator.Builder().commitCount(2).build()); + } + + private static void assertTriggers(List<Trigger> expected, List<Trigger> actual) { + assertThat(actual).hasSize(expected.size()); + for (int i = 0; i < expected.size(); ++i) { + Trigger expectedTrigger = expected.get(i); + Trigger actualTrigger = actual.get(i); + assertThat(actualTrigger.timestamp()).isEqualTo(expectedTrigger.timestamp()); + assertThat(actualTrigger.taskId()).isEqualTo(expectedTrigger.taskId()); + assertThat(actualTrigger.isRecovery()).isEqualTo(expectedTrigger.isRecovery()); + if (expectedTrigger.table() == null) { + assertThat(actualTrigger.table()).isNull(); + } else { + Iterator<Snapshot> expectedSnapshots = expectedTrigger.table().snapshots().iterator(); + Iterator<Snapshot> actualSnapshots = actualTrigger.table().snapshots().iterator(); + while (expectedSnapshots.hasNext()) { + assertThat(actualSnapshots.hasNext()).isTrue(); + assertThat(expectedSnapshots.next().snapshotId()) + .isEqualTo(actualSnapshots.next().snapshotId()); + } + } + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List<StreamElement> transformsToStreamElement(Collection<Object> elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord<CommittableMessage<?>>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary<?> extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary<?>) value; + } + + static CommittableWithLineage<IcebergCommittable> extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage<IcebergCommittable>) value; + } +} 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 59% 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 index 14f12422da96..3299e7a97776 100644 --- 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 @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; @@ -33,55 +40,50 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; - private final Map<String, String> initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } + @Parameter(index = 0) + private Map<String, String> initProperties; - public TestCompressionSettings(Map<String, String> initProperties) { - this.initProperties = initProperties; + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; } - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); } - @Test + @TestTemplate public void testCompressionAvro() throws Exception { // No override provided Map<String, String> resultProperties = @@ -91,19 +93,17 @@ public void testCompressionAvro() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -119,11 +119,12 @@ public void testCompressionAvro() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionParquet() throws Exception { // No override provided Map<String, String> resultProperties = @@ -133,19 +134,19 @@ public void testCompressionParquet() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -161,11 +162,12 @@ public void testCompressionParquet() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionOrc() throws Exception { // No override provided Map<String, String> resultProperties = @@ -175,19 +177,18 @@ public void testCompressionOrc() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); } // Override compression to snappy and a different strategy @@ -203,12 +204,15 @@ public void testCompressionOrc() throws Exception { FlinkWriteOptions.COMPRESSION_STRATEGY.key(), "speed")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map<String, String> override) throws Exception { + private static OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map<String, String> override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -216,7 +220,7 @@ private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIce IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness<RowData, WriteResult> harness = + OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -227,7 +231,7 @@ private static OneInputStreamOperatorTestHarness<RowData, WriteResult> createIce private static Map<String, String> appenderProperties( Table table, TableSchema schema, Map<String, String> override) throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); 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<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream<RowData> 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> 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<Row> createBoundedSource(List<Row> rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List<Row> createRows(String prefix) { protected List<RowData> convertToRowData(List<Row> rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> 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..df8c3c79d3e3 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,494 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +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<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> 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 testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // sort based on partition columns + builder.append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + } + + @TestTemplate + public void testRangeDistributionWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream<Row> dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List<Row> rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream<Row> dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the newest snapshot to the oldest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Sometimes we will have more checkpoints than the bounded source if we pass the + // auto checkpoint interval. Thus producing multiple snapshots. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(numOfCheckpoints); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // each writer task should only write one file for non-partition sort column + // sometimes + assertThat(addedDataFiles).hasSize(parallelism); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource<Row> createRangeDistributionBoundedSource( + List<List<Row>> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List<List<Row>> createCharRows(int numOfCheckpoints, int countPerChar) { + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List<Row> rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List<List<Row>> createIntRows(int numOfCheckpoints, int maxId) { + List<List<Row>> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List<Row> rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/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<String, String> 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<Row> leftRows = createRows("left-"); + DataStream<Row> 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<Row> rightRows = createRows("right-"); + DataStream<Row> 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<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> 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<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream<RowData> 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.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java new file mode 100644 index 000000000000..a5f24e09a60b --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test range distribution with bucketing partition column. Compared to hash distribution, range + * distribution is more general to handle bucketing column while achieving even distribution of + * traffic to writer tasks. + * + * <ul> + * <li><a href="https://github.com/apache/iceberg/pull/4228">keyBy on low cardinality</a> (e.g. + * 60) may not achieve balanced data distribution. + * <li>number of buckets (e.g. 60) is not divisible by the writer parallelism (e.g. 40). + * <li>number of buckets (e.g. 60) is smaller than the writer parallelism (e.g. 120). + * </ul> + */ +@Timeout(value = 30) +public class TestFlinkIcebergSinkRangeDistributionBucketing { + private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + // max supported parallelism is 16 (= 4 x 4) + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(4) + .setNumberSlotsPerTaskManager(4) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private static final int NUM_BUCKETS = 4; + private static final int NUM_OF_CHECKPOINTS = 4; + private static final int ROW_COUNT_PER_CHECKPOINT = 200; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "uuid", Types.UUIDType.get()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("uuid", NUM_BUCKETS).build(); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + private TableLoader tableLoader; + private Table table; + + @BeforeEach + public void before() throws IOException { + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name())); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + // Assuming ts is on ingestion/processing time. Writer only writes to 1 or 2 hours concurrently. + // Only sort on the bucket column to avoid each writer task writes to 60 buckets/files + // concurrently. + table.replaceSortOrder().asc(bucket("uuid", NUM_BUCKETS)).commit(); + } + + @AfterEach + public void after() throws Exception { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + /** number of buckets 4 matches writer parallelism of 4 */ + @Test + public void testBucketNumberEqualsToWriterParallelism() throws Exception { + testParallelism(4); + } + + /** number of buckets 4 is less than writer parallelism of 6 */ + @Test + public void testBucketNumberLessThanWriterParallelismNotDivisible() throws Exception { + testParallelism(6); + } + + /** number of buckets 4 is less than writer parallelism of 8 */ + @Test + public void testBucketNumberLessThanWriterParallelismDivisible() throws Exception { + testParallelism(8); + } + + /** number of buckets 4 is greater than writer parallelism of 3 */ + @Test + public void testBucketNumberHigherThanWriterParallelismNotDivisible() throws Exception { + testParallelism(3); + } + + /** number of buckets 4 is greater than writer parallelism of 2 */ + @Test + public void testBucketNumberHigherThanWriterParallelismDivisible() throws Exception { + testParallelism(2); + } + + private void testParallelism(int parallelism) throws Exception { + try (StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism)) { + + DataGeneratorSource<RowData> generatorSource = + new DataGeneratorSource<>( + new RowGenerator(), + ROW_COUNT_PER_CHECKPOINT * NUM_OF_CHECKPOINTS, + RateLimiterStrategy.perCheckpoint(ROW_COUNT_PER_CHECKPOINT), + FlinkCompatibilityUtil.toTypeInfo(ROW_TYPE)); + DataStream<RowData> dataStream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the oldest snapshot to the newest snapshot + List<Snapshot> snapshots = Lists.newArrayList(table.snapshots().iterator()); + // only keep the snapshots with added data files + snapshots = + snapshots.stream() + .filter(snapshot -> snapshot.addedDataFiles(table.io()).iterator().hasNext()) + .collect(Collectors.toList()); + + // Source rate limit per checkpoint cycle may not be super precise. + // There could be more checkpoint cycles and commits than planned. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(NUM_OF_CHECKPOINTS); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List<Snapshot> rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List<DataFile> addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + assertThat(addedDataFiles) + .hasSizeLessThanOrEqualTo(maxAddedDataFilesPerCheckpoint(parallelism)); + } + } + } + + /** + * Traffic is not perfectly balanced across all buckets in the small sample size Range + * distribution of the bucket id may cross subtask boundary. Hence the number of committed data + * files per checkpoint maybe larger than writer parallelism or the number of buckets. But it + * should not be more than the sum of those two. Without range distribution, the number of data + * files per commit can be 4x of parallelism (as the number of buckets is 4). + */ + private int maxAddedDataFilesPerCheckpoint(int parallelism) { + return NUM_BUCKETS + parallelism; + } + + private static class RowGenerator implements GeneratorFunction<Long, RowData> { + // use constant timestamp so that all rows go to the same hourly partition + private final long ts = System.currentTimeMillis(); + + @Override + public RowData map(Long index) throws Exception { + // random uuid should result in relatively balanced distribution across buckets + UUID uuid = UUID.randomUUID(); + ByteBuffer uuidByteBuffer = ByteBuffer.allocate(16); + uuidByteBuffer.putLong(uuid.getMostSignificantBits()); + uuidByteBuffer.putLong(uuid.getLeastSignificantBits()); + return GenericRowData.of( + TimestampData.fromEpochMillis(ts), + uuidByteBuffer.array(), + StringData.fromString("row-" + index)); + } + } +} diff --git a/flink/v1.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 89% 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..b283b8390a2b 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 @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -38,7 +39,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 +57,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 +89,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -185,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.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 90% 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 index fc33c2fea5e6..da4ac2ada677 100644 --- 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 @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation<Row> ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation<Row> ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map<String, RowKind> ROW_KIND_MAP = + static final Map<String, RowKind> ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List<List<Row>> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List<String> equalityFieldColumns, KeySelector<Row, Object> keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List<Snapshot> findValidSnapshots() { + List<Snapshot> findValidSnapshots() { List<Snapshot> validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List<Snapshot> findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable<Record> reader = 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 95% 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..a2e9be9303c3 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; @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -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 85% 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 index ce1f208a4b07..c21c3d5cc21b 100644 --- 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 @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -45,28 +47,26 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkManifest { private static final Configuration CONF = new Configuration(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; private FileAppenderFactory<RowData> appenderFactory; private final AtomicInteger fileCount = new AtomicInteger(0); - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + assertThat(new File(tablePath).mkdir()).isTrue(); // Construct the iceberg table. table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); @@ -112,11 +112,11 @@ public void testIO() throws IOException { WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < 5; i++) { TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); } @@ -131,9 +131,11 @@ public void testUserProvidedManifestLocation() throws IOException { long checkpointId = 1; String flinkJobId = newFlinkJobId(); String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map<String, String> props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); @@ -144,21 +146,18 @@ public void testUserProvidedManifestLocation() throws IOException { () -> factory.create(checkpointId), table.spec()); - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } @@ -198,7 +197,7 @@ public void testVersionedSerializer() throws IOException { byte[] versionedSerializeData2 = SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); } @Test @@ -220,14 +219,13 @@ public void testCompatibility() throws IOException { DeltaManifests delta = SimpleVersionedSerialization.readVersionAndDeSerialize( DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); TestHelpers.assertEquals(manifest, delta.dataManifest()); List<DataFile> actualFiles = FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); + assertThat(actualFiles).hasSize(10); for (int i = 0; i < 10; i++) { TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/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<T> extends TestPartitioningWriters<RowData> { + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List<Object> parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); + } + @Override protected FileWriterFactory<RowData> 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.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..abdb2b5c79ae --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List<Object> parameters() { + List<Object> parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map<String, String> props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest<IcebergCommittable> commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest<IcebergCommittable> commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List<RowData> rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest<IcebergCommittable> commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map<String, String> summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary<IcebergCommittable> committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List<StreamElement> output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary<IcebergCommittable> committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List<StreamElement> output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary<IcebergCommittable> committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary<IcebergCommittable> committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List<StreamElement> output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary<IcebergCommittable> committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary<IcebergCommittable> committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List<StreamElement> output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary<IcebergCommittable> committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List<StreamElement> output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List<RowData> expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List<RowData> rows = Lists.newArrayList(); + List<RowData> tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List<RowData> tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List<RowData> rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List<RowData> expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List<Path> manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List<DataFile> dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List<StreamElement> output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List<StreamElement> output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List<StreamElement> output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List<StreamElement> output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary<IcebergCommittable> processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary<IcebergCommittable> committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage<IcebergCommittable> committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary<IcebergCommittable> processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory<RowData> createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List<Path> assertFlinkManifests(int expectedCount) throws IOException { + List<Path> manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List<RowData> rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory<RowData> appenderFactory, String filename, List<RowData> deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage<IcebergCommittable>, CommittableMessage<IcebergCommittable>> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest<IcebergCommittable> buildCommitRequestFor( + String myJobID, long checkpoint, Collection<WriteResult> writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest<IcebergCommittable> commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer<CommittableMessage<IcebergCommittable>> { + + CommittableMessageSerializer<IcebergCommittable> serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer<CommittableMessage<IcebergCommittable>> duplicate() { + return null; + } + + @Override + public CommittableMessage<IcebergCommittable> createInstance() { + return null; + } + + @Override + public CommittableMessage<IcebergCommittable> copy( + CommittableMessage<IcebergCommittable> from) { + return from; + } + + @Override + public CommittableMessage<IcebergCommittable> copy( + CommittableMessage<IcebergCommittable> from, CommittableMessage<IcebergCommittable> reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage<IcebergCommittable> record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage<IcebergCommittable> deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage<IcebergCommittable> deserialize( + CommittableMessage<IcebergCommittable> reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage<IcebergCommittable> deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot<CommittableMessage<IcebergCommittable>> snapshotConfiguration() { + return null; + } + }; +} 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 82% 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 index 948c7b31430c..7808771d9887 100644 --- 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 @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -139,8 +140,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -161,7 +161,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +177,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +194,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +206,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +235,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +246,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +289,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +300,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +341,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +354,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +367,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +381,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +406,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List<RowData> expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +418,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +454,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +481,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +504,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +518,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +536,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +550,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +575,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +588,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +612,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness<WriteResult, Void> harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +631,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +654,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness<WriteResult, Void> harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +681,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +707,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +720,14 @@ public void testBoundedStream() throws Exception { List<RowData> tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +740,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +751,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +791,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +801,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +833,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +865,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory<RowData> appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +880,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +894,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +911,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + * <ul> + * <li>A specific row is updated + * <li>The prepareSnapshotPreBarrier triggered + * <li>Checkpoint failed for reasons outside of the Iceberg connector + * <li>The specific row is updated again in the second checkpoint as well + * <li>Second snapshot is triggered, and finished + * </ul> + * + * <p>Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory<RowData> appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +996,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1008,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1027,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1045,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness<WriteResult, Void> harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1062,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1079,8 +1178,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } @@ -1089,7 +1188,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness<WriteResult, Void> createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness<FlinkWriteResult, Void> createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1208,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory<Void> - implements OneInputStreamOperatorFactory<WriteResult, Void> { + implements OneInputStreamOperatorFactory<FlinkWriteResult, Void> { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java new file mode 100644 index 000000000000..7f355c1e8403 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -0,0 +1,436 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.IcebergSink.Builder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSink extends TestFlinkIcebergSinkBase { + + private TableLoader tableLoader; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + void testWriteRowData() throws Exception { + List<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream<RowData> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @TestTemplate + void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @TestTemplate + void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); + } + } + } + + @TestTemplate + void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @TestTemplate + void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testTwoSinksInDisjointedDAG() throws Exception { + Map<String, String> 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<Row> leftRows = createRows("left-"); + DataStream<Row> leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidSuffix("leftIcebergSink") + .append(); + + List<Row> rightRows = createRows("right-"); + DataStream<Row> rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + IcebergSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownDistributionMode() { + Map<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); + + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownFileFormat() { + Map<String, String> newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List<Row> rows = createRows(""); + DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .uidSuffix("ingestion"); + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @TestTemplate + void testWriteRowWithTableRefreshInterval() throws Exception { + List<Row> rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream<RowData> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List<Row> rows = createRows(""); + DataStream<Row> dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream<Row> dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List<List<Row>> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List<List<Record>> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream<Row> dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List<List<Row>> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List<List<Record>> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + new String( + deleteFile.lowerBounds().get(MetadataColumns.DELETE_FILE_PATH.fieldId()).array()); + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + assumeThat(fromStat).isEqualTo(dataFile.path().toString()); + } + + protected void testChangeLogs( + List<String> equalityFieldColumns, + KeySelector<Row, Object> keySelector, + boolean insertAsUpsert, + List<List<Row>> elementsPerCheckpoint, + List<List<Record>> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream<Row> dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List<Snapshot> snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List<Record> expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java new file mode 100644 index 000000000000..93da40ab5c9a --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Branch { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} 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 68% 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 index c05f08b82f51..e13721a9f170 100644 --- 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 @@ -22,11 +22,13 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -42,6 +44,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -58,52 +63,47 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; private Table table; - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table. Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,10 +111,11 @@ public void testWritingTable() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); checkpointId = checkpointId + 1; @@ -124,9 +125,10 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); @@ -145,49 +147,53 @@ public void testWritingTable() throws Exception { } } - @Test + @TestTemplate public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } } - @Test + @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); + assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); + assertThat(scanDataFiles()).hasSize(1); } private Set<String> scanDataFiles() throws IOException { @@ -211,9 +217,9 @@ private Set<String> scanDataFiles() throws IOException { } } - @Test + @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,45 +227,49 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); testHarness.endInput(); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testTableWithTargetFileSize() throws Exception { // Adjust the target-file-size in table properties. table @@ -276,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -284,13 +294,14 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); // Assert that the data file have the expected records. for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); + assertThat(dataFile.recordCount()).isEqualTo(1000); } // Commit the iceberg transaction. @@ -303,7 +314,7 @@ public void testTableWithTargetFileSize() throws Exception { SimpleDataUtil.assertTableRecords(table, records); } - @Test + @TestTemplate public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( @@ -329,7 +340,8 @@ public void testPromotedFlinkDataType() throws Exception { spec = PartitionSpec.unpartitioned(); } - String location = tempFolder.newFolder().getAbsolutePath(); + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); @@ -346,15 +358,16 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness<RowData, WriteResult> testHarness = + try (OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter() + private static List<WriteResult> getWriteResults(List<FlinkWriteResult> flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness<RowData, WriteResult> createIcebergStr IcebergStreamWriter<RowData> streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness<RowData, WriteResult> harness = + OneInputStreamOperatorTestHarness<RowData, FlinkWriteResult> harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); 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 81% 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 index 79be96d20902..919fef579ab0 100644 --- 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +37,7 @@ import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRowDataPartitionKey { private static final Schema SCHEMA = @@ -91,10 +92,10 @@ public void testNullPartitionValue() { for (RowData row : rows) { PartitionKey partitionKey = new PartitionKey(spec, schema); partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); + assertThat(partitionKey.size()).isEqualTo(1); String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); } } @@ -116,15 +117,15 @@ public void testPartitionWithOneNestedField() { PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); + assertThat(partitionKey1.size()).isEqualTo(1); - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); + assertThat(partitionKey2.size()).isEqualTo(1); - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); } } @@ -154,16 +155,16 @@ public void testPartitionMultipleNestedField() { Record record = (Record) records.get(i).get(0); pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); + assertThat(pk1.size()).isEqualTo(2); - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); + assertThat(pk2.size()).isEqualTo(2); - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); } } @@ -190,19 +191,18 @@ public void testPartitionValueTypes() { pk.partition(rowWrapper.wrap(row)); expectedPK.partition(recordWrapper.wrap(record)); - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } @@ -232,19 +232,18 @@ public void testNestedPartitionValues() { pk.partition(rowWrapper.wrap(rows.get(j))); expectedPK.partition(recordWrapper.wrap(records.get(j))); - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java similarity index 80% 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 index 112dbb511310..8bfd6cb3d043 100644 --- 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 @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; @@ -30,6 +33,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -39,69 +45,62 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestTaskWriters { private static final Configuration CONF = new Configuration(); private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; - private Table table; + @Parameter(index = 1) + private boolean partitioned; - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } + private Table table; - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table with the specified file format. Map<String, String> props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); // Close again. taskWriter.close(); dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); } } - @Test + @TestTemplate public void testCloseTwice() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -111,16 +110,16 @@ public void testCloseTwice() throws IOException { int expectedFiles = partitioned ? 2 : 1; DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } } } - @Test + @TestTemplate public void testAbort() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -130,16 +129,16 @@ public void testAbort() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); } } } - @Test + @TestTemplate public void testCompleteFiles() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "a")); @@ -149,14 +148,14 @@ public void testCompleteFiles() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); @@ -176,7 +175,7 @@ public void testCompleteFiles() throws IOException { } } - @Test + @TestTemplate public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(4)) { List<RowData> rows = Lists.newArrayListWithCapacity(8000); @@ -193,7 +192,7 @@ public void testRollingWithTargetFileSize() throws IOException { } DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); + assertThat(dataFiles).hasSize(8); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { @@ -206,7 +205,7 @@ public void testRollingWithTargetFileSize() throws IOException { } } - @Test + @TestTemplate public void testRandomData() throws IOException { try (TaskWriter<RowData> taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable<RowData> rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..5910bd685510 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +class Fixtures { + private Fixtures() {} + + public static final int NUM_SUBTASKS = 2; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); + public static final TypeSerializer<RowData> ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); + public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + public static final Comparator<StructLike> SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); + public static final SortKeySerializer SORT_KEY_SERIALIZER = + new SortKeySerializer(SCHEMA, SORT_ORDER); + public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = + new DataStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); + + public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + public static final Map<String, SortKey> CHAR_KEYS = createCharKeys(); + + public static StatisticsEvent createStatisticsEvent( + StatisticsType type, + TypeSerializer<DataStatistics> statisticsSerializer, + long checkpointId, + SortKey... keys) { + DataStatistics statistics = createTaskStatistics(type, keys); + return StatisticsEvent.createTaskStatisticsEvent( + checkpointId, statistics, statisticsSerializer); + } + + public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { + DataStatistics statistics; + if (type == StatisticsType.Sketch) { + statistics = new SketchDataStatistics(128); + } else { + statistics = new MapDataStatistics(); + } + + for (SortKey key : keys) { + statistics.add(key); + } + + return statistics; + } + + private static Map<String, SortKey> createCharKeys() { + Map<String, SortKey> keys = Maps.newHashMap(); + for (char c = 'a'; c <= 'z'; ++c) { + String key = Character.toString(c); + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, key); + keys.put(key, sortKey); + } + + return keys; + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..8322ce683768 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -0,0 +1,465 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestAggregatedStatisticsTracker { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveNewerStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + // checkpoint 1 is completed + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + // checkpoint 2 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveOlderStatisticsEventTest(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint3Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); + // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 4L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveCompletedStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + + // Receive data statistics from all subtasks at checkpoint 1 + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 3L, + CHAR_KEYS.get("b"), 3L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint2Subtask0DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); + completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask1DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); + // Receive data statistics from all subtasks at checkpoint 2 + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + } + + @Test + public void coordinatorSwitchToSketchOverThreshold() { + int parallelism = 3; + int downstreamParallelism = 3; + int switchToSketchThreshold = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + switchToSketchThreshold, + null); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + // converted to sketch statistics as map size is 4 (over the switch threshold of 3) + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + @Test + public void coordinatorMapOperatorSketch() { + int parallelism = 3; + int downstreamParallelism = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + + // first operator event has map statistics + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + // second operator event contains sketch statistics + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Sketch, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + // third operator event has Map statistics + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + private AggregatedStatisticsTracker createTracker(StatisticsType type) { + return new AggregatedStatisticsTracker( + "testOperator", + Fixtures.NUM_SUBTASKS, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java new file mode 100644 index 000000000000..4ee9888934a8 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestCompletedStatisticsSerializer extends SerializerTestBase<CompletedStatistics> { + + @Override + protected TypeSerializer<CompletedStatistics> createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<CompletedStatistics> getTypeClass() { + return CompletedStatistics.class; + } + + @Override + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( + 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java new file mode 100644 index 000000000000..a08a8a73e80c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestDataStatisticsCoordinator { + private static final String OPERATOR_NAME = "TestCoordinator"; + private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); + + private EventReceivingTasks receivingTasks; + + @BeforeEach + public void before() throws Exception { + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + private void tasksReady(DataStatisticsCoordinator coordinator) { + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + StatisticsEvent.createTaskStatisticsEvent( + 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + Map<SortKey, Long> keyFrequency = + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 3L, + CHAR_KEYS.get("c"), 5L); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(NUM_SUBTASKS, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.checkpointId()).isEqualTo(1L); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } + } + + @Test + public void testRequestGlobalStatisticsEventHandling() throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); + } + } + + static void setAllTasksReady( + int subtasks, + DataStatisticsCoordinator dataStatisticsCoordinator, + EventReceivingTasks receivingTasks) { + for (int i = 0; i < subtasks; i++) { + dataStatisticsCoordinator.executionAttemptReady( + i, 0, receivingTasks.createGatewayForSubtask(i, 0)); + } + } + + static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { + CompletableFuture<Void> future = new CompletableFuture<>(); + coordinator.callInCoordinatorThread( + () -> { + future.complete(null); + return null; + }, + "Coordinator fails to process action"); + + try { + future.get(); + } catch (InterruptedException e) { + throw new AssertionError("test interrupted"); + } catch (ExecutionException e) { + ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); + } + } + + private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { + return new DataStatisticsCoordinator( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + NUM_SUBTASKS, + type, + 0.0d); + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..6317f2bfde18 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestDataStatisticsCoordinatorProvider { + private static final OperatorID OPERATOR_ID = new OperatorID(); + + private EventReceivingTasks receivingTasks; + + @BeforeEach + public void before() { + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testCheckpointAndReset(StatisticsType type) throws Exception { + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); + try (RecreateOnResetOperatorCoordinator coordinator = + (RecreateOnResetOperatorCoordinator) + provider.create( + new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { + DataStatisticsCoordinator dataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + + // Handle events from operators for checkpoint 1 + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + // Verify checkpoint 1 global data statistics + Map<SortKey, Long> checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); + coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + // Verify checkpoint 2 global data statistics + Map<SortKey, Long> checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + } + + waitForCheckpoint(2L, dataStatisticsCoordinator); + + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + // Verify restored data statistics + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + } + } + + private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) + throws InterruptedException, ExecutionException { + CompletableFuture<byte[]> future = new CompletableFuture<>(); + coordinator.checkpointCoordinator(checkpointId, future); + return future.get(); + } + + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { + return new DataStatisticsCoordinatorProvider( + "DataStatisticsCoordinatorProvider", + OPERATOR_ID, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + type, + 0.0); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java new file mode 100644 index 000000000000..bc248b778184 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -0,0 +1,352 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; + +public class TestDataStatisticsOperator { + + private Environment env; + + @BeforeEach + public void before() throws Exception { + this.env = + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); + } + + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { + MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + return createOperator(type, downstreamParallelism, mockGateway); + } + + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { + DataStatisticsOperator operator = + new DataStatisticsOperator( + "testOperator", + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + mockGateway, + downstreamParallelism, + type); + operator.setup( + new OneInputStreamTask<String, String>(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + return operator; + } + + @SuppressWarnings("unchecked") + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElement(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); + + DataStatistics localStatistics = operator.localStatistics(); + assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + Map<SortKey, Long> keyFrequency = (Map<SortKey, Long>) localStatistics.result(); + assertThat(keyFrequency) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); + } else { + ReservoirItemsSketch<SortKey> sketch = + (ReservoirItemsSketch<SortKey>) localStatistics.result(); + assertThat(sketch.getSamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + testHarness.endInput(); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testOperatorOutput(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); + + List<RowData> recordsOutput = + testHarness.extractOutputValues().stream() + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record) + .collect(Collectors.toList()); + assertThat(recordsOutput) + .containsExactlyInAnyOrderElementsOf( + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); + } + } + + private static Stream<Arguments> provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + + @ParameterizedTest + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { + Map<SortKey, Long> keyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); + SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + OperatorSubtaskState snapshot; + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness1 = + createHarness(operator)) { + GlobalStatistics statistics; + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); + } else { + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); + } + + StatisticsEvent event = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); + operator.handleOperatorEvent(event); + + GlobalStatistics globalStatistics = operator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + + snapshot = testHarness1.snapshot(1L, 0); + } + + // Use the snapshot to initialize state for another new operator and then verify that the global + // statistics for the new operator is same as before + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + testHarness2.setup(); + testHarness2.initializeState(snapshot); + + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithLocalStatsOverThreshold() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // add rows with unique keys + for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map<SortKey, Long>) operator.localStatistics().result()).hasSize(i + 1); + } + + // one more item should trigger the migration to sketch stats + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch<SortKey> sketch = + (ReservoirItemsSketch<SortKey>) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); + // reservoir not full yet + assertThat(sketch.getN()).isLessThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); + + // add more items to saturate the reservoir + for (int i = 0; i < reservoirSize; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + } + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + sketch = (ReservoirItemsSketch<SortKey>) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()) + .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); + // reservoir is full now + assertThat(sketch.getN()).isGreaterThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize(reservoirSize); + + testHarness.endInput(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithGlobalSketchStatistics() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // started with Map stype + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map<SortKey, Long>) operator.localStatistics().result()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + + // received global statistics with sketch type + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( + 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); + operator.handleOperatorEvent( + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch<SortKey> sketch = + (ReservoirItemsSketch<SortKey>) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(1); + assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); + + testHarness.endInput(); + } + } + + private StateInitializationContext getStateContext() throws Exception { + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + new OperatorStateBackendParametersImpl( + env, "test-operator", Collections.emptyList(), cancelStreamRegistry)); + return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + } + + private OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> createHarness( + DataStatisticsOperator dataStatisticsOperator) throws Exception { + OneInputStreamOperatorTestHarness<RowData, StatisticsOrRecord> harness = + new OneInputStreamOperatorTestHarness<>( + dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); + harness.setup( + new StatisticsOrRecordSerializer( + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); + harness.open(); + return harness; + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..59ce6df05d9d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class TestDataStatisticsSerializer extends SerializerTestBase<DataStatistics> { + @Override + protected TypeSerializer<DataStatistics> createSerializer() { + return Fixtures.TASK_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<DataStatistics> getTypeClass() { + return DataStatistics.class; + } + + @Override + protected DataStatistics[] getTestData() { + return new DataStatistics[] { + new MapDataStatistics(), + Fixtures.createTaskStatistics( + StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), + new SketchDataStatistics(128), + Fixtures.createTaskStatistics( + StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) + }; + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase<GlobalStatistics> { + + @Override + protected TypeSerializer<GlobalStatistics> createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class<GlobalStatistics> getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.20/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 new file mode 100644 index 000000000000..8a25c7ad9898 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("a")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + Map<SortKey, Long> actual = (Map<SortKey, Long>) dataStatistics.result(); + Map<SortKey, Long> expected = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java similarity index 62% 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 index e6726e7db785..d5a0bebc74e7 100644 --- 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -64,65 +65,60 @@ private static SortKey[] initSortKeys() { } // Total weight is 800 - private final MapDataStatistics mapDataStatistics = - new MapDataStatistics( - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L)); + private final Map<SortKey, Long> mapStatistics = + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L); @Test public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // each task should get targeted weight of 100 (=800/8) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -144,19 +140,20 @@ public void testEvenlyDividableNoClosingFileCost() { Pair.of(100L, 1), 7, Pair.of(100L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 100 before close file cost factored in. // close file cost is 5 = 5% * 100. @@ -165,35 +162,30 @@ public void testEvenlyDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight with close cost per subtask is 110 (880/8) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -216,51 +208,48 @@ public void testEvenlyDividableWithClosingFileCost() { Pair.of(100L, 2), 7, Pair.of(75L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 // each task should get targeted weight of 89 = ceiling(800/9) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -284,19 +273,20 @@ public void testNonDividableNoClosingFileCost() { Pair.of(89L, 2), 8, Pair.of(88L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 89 before close file cost factored in. // close file cost is 5 (= 5% * 89) per file. @@ -305,35 +295,31 @@ public void testNonDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight per subtask is 98 ceiling(880/9) - Map<SortKey, MapRangePartitioner.KeyAssignment> expectedAssignment = + Map<SortKey, KeyAssignment> expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map<SortKey, MapRangePartitioner.KeyAssignment> actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -358,40 +344,39 @@ public void testNonDividableWithClosingFileCost() { Pair.of(88L, 2), 8, Pair.of(61L, 7)); - Map<Integer, Pair<Long, Integer>> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); // drift threshold is high for non-dividable scenario with close cost validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); } private static Map<Integer, Pair<AtomicLong, Set<RowData>>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { + MapRangePartitioner partitioner, int numPartitions, Map<SortKey, Long> mapStatistics) { // The Map key is the subtaskId. // For the map value pair, the first element is the count of assigned and // the second element of Set<String> is for the set of assigned keys. Map<Integer, Pair<AtomicLong, Set<RowData>>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair<AtomicLong, Set<RowData>> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair<AtomicLong, Set<RowData>> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); return partitionResults; } 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<Integer> 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<Integer> 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.20/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 new file mode 100644 index 000000000000..396bfae2f13c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + SketchDataStatistics dataStatistics = new SketchDataStatistics(128); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + ReservoirItemsSketch<SortKey> actual = (ReservoirItemsSketch<SortKey>) dataStatistics.result(); + assertThat(actual.getSamples()) + .isEqualTo( + new SortKey[] { + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") + }); + } +} diff --git a/flink/v1.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 + * + * <ul> + * <li>age <= 15 + * <li>age > 15 && age <= 32 + * <li>age >32 && age <= 60 + * <li>age > 60 + * </ul> + */ + 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.20/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 new file mode 100644 index 000000000000..16202c075ea0 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSketchUtil { + @Test + public void testCoordinatorReservoirSize() { + // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); + // adjust to multiplier of 100 + assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); + // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) + .isEqualTo(1_000_000 - (1_000_000 % 10_123)); + } + + @Test + public void testOperatorReservoirSize() { + assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) + .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); + assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) + .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); + assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) + .isEqualTo( + (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); + assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) + .isEqualTo( + (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); + } + + @Test + public void testRangeBoundsOneChannel() { + assertThat( + SketchUtil.rangeBounds( + 1, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .isEmpty(); + } + + @Test + public void testRangeBoundsDivisible() { + assertThat( + SketchUtil.rangeBounds( + 3, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void testRangeBoundsNonDivisible() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); + } + + @Test + public void testRangeBoundsSkipDuplicates() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + // skipped duplicate c's + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); + } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } +} diff --git a/flink/v1.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 61% 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 index 291302aef486..54cceae6e55b 100644 --- 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 @@ -18,14 +18,24 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.iceberg.NullOrder; import org.apache.iceberg.Schema; import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.RowDataWrapper; +import org.junit.jupiter.api.Test; public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { private final DataGenerator generator = new DataGenerators.Primitives(); @@ -54,4 +64,27 @@ protected SortOrder sortOrder() { protected GenericRowData rowData() { return generator.generateFlinkRowData(); } + + @Test + public void testSerializationSize() throws Exception { + RowData rowData = + GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = Fixtures.SORT_KEY.copy(); + sortKey.wrap(struct); + SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataOutputSerializer output = new DataOutputSerializer(1024); + serializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + assertThat(serializedBytes.length) + .as( + "Serialized bytes for sort key should be 38 bytes (34 UUID text + 4 byte integer of string length") + .isEqualTo(38); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = serializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..c0f688f2589e --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() throws Exception { + RowData rowData = GenericRowData.of(StringData.fromString("str"), 1); + RowDataWrapper rowDataWrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(struct); + + SortKeySerializer originalSerializer = new SortKeySerializer(SCHEMA, SORT_ORDER); + TypeSerializerSnapshot<SortKey> snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer<SortKey> restoredSerializer = snapshot.restoreSerializer(); + + DataOutputSerializer output = new DataOutputSerializer(1024); + originalSerializer.serialize(sortKey, output); + byte[] serializedBytes = output.getCopyOfBuffer(); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = restoredSerializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } + + @Test + public void testSnapshotIsCompatibleWithSameSortOrder() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility<SortKey> resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot<SortKey> original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} diff --git a/flink/v1.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.20/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 new file mode 100644 index 000000000000..f89d63ac73e3 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.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<RowData> writeAndGetAppender(List<Record> 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<RowData> 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 86% 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 bc7194e38088..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 @@ -24,12 +24,12 @@ import java.util.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestFlinkSourceConfig extends TestFlinkTableSource { +public class TestFlinkSourceConfig extends TableSourceTestBase { private static final String TABLE = "test_table"; - @Test + @TestTemplate public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) @@ -37,7 +37,7 @@ public void testFlinkSessionConfig() { .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } - @Test + @TestTemplate public void testFlinkHintConfig() { List<Row> result = sql( @@ -46,10 +46,13 @@ public void testFlinkHintConfig() { assertThat(result).hasSize(3); } - @Test + @TestTemplate public void testReadOptionHierarchy() { getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List<Row> 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 89% 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 index 01bab6d063fd..18528c789114 100644 --- 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 @@ -21,78 +21,16 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; -import java.io.IOException; import java.util.List; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestFlinkTableSource extends TestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - } - - @Test +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate public void testLimitPushDown() { assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) @@ -121,7 +59,7 @@ public void testLimitPushDown() { assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } - @Test + @TestTemplate public void testNoFilterPushDown() { String sql = String.format("SELECT * FROM %s ", TABLE_NAME); List<Row> result = sql(sql); @@ -133,7 +71,7 @@ public void testNoFilterPushDown() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDownEqual() { String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -147,7 +85,7 @@ public void testFilterPushDownEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); @@ -156,7 +94,7 @@ public void testFilterPushDownEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownEqualLiteralOnLeft() { String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -170,7 +108,7 @@ public void testFilterPushDownEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqual() { String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") != 1"; @@ -187,7 +125,7 @@ public void testFilterPushDownNoEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); @@ -196,7 +134,7 @@ public void testFilterPushDownNoEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownAnd() { String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); @@ -211,7 +149,7 @@ public void testFilterPushDownAnd() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownOr() { String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; @@ -229,7 +167,7 @@ public void testFilterPushDownOr() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThan() { String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 1"; @@ -247,7 +185,7 @@ public void testFilterPushDownGreaterThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); @@ -256,7 +194,7 @@ public void testFilterPushDownGreaterThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanLiteralOnLeft() { String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 3"; @@ -274,7 +212,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqual() { String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 2"; @@ -292,7 +230,7 @@ public void testFilterPushDownGreaterThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); @@ -301,7 +239,7 @@ public void testFilterPushDownGreaterThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 2"; @@ -319,7 +257,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThan() { String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 2"; @@ -334,7 +272,7 @@ public void testFilterPushDownLessThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); @@ -343,7 +281,7 @@ public void testFilterPushDownLessThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanLiteralOnLeft() { String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 2"; @@ -358,7 +296,7 @@ public void testFilterPushDownLessThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqual() { String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 1"; @@ -373,7 +311,7 @@ public void testFilterPushDownLessThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); @@ -382,7 +320,7 @@ public void testFilterPushDownLessThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualLiteralOnLeft() { String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 3"; @@ -397,7 +335,7 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; @@ -413,7 +351,7 @@ public void testFilterPushDownIn() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownInNull() { String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); @@ -430,7 +368,7 @@ public void testFilterPushDownInNull() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); @@ -444,7 +382,7 @@ public void testFilterPushDownNotIn() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List<Row> resultGT = sql(sqlNotInNull); @@ -455,7 +393,7 @@ public void testFilterPushDownNotInNull() { .isNull(); } - @Test + @TestTemplate public void testFilterPushDownIsNotNull() { String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); String expectedFilter = "not_null(ref(name=\"data\"))"; @@ -473,7 +411,7 @@ public void testFilterPushDownIsNotNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIsNull() { String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); String expectedFilter = "is_null(ref(name=\"data\"))"; @@ -488,7 +426,7 @@ public void testFilterPushDownIsNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); @@ -503,7 +441,7 @@ public void testFilterPushDownNot() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); @@ -522,7 +460,7 @@ public void testFilterPushDownBetween() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownNotBetween() { String sqlNotBetween = String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); @@ -538,7 +476,7 @@ public void testFilterPushDownNotBetween() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLike() { String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; @@ -565,7 +503,7 @@ public void testFilterPushDownLike() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; @@ -604,7 +542,7 @@ public void testFilterNotPushDownLike() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDown2Literal() { String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); List<Row> result = sql(sql2Literal); @@ -616,7 +554,7 @@ public void testFilterPushDown2Literal() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testSqlParseNaN() { // todo add some test case to test NaN } diff --git a/flink/v1.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 94% 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 index b7447d15c05a..db8647f054ae 100644 --- 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 @@ -24,8 +24,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -130,11 +128,8 @@ protected List<Row> run( sourceBuilder.properties(options); DataStream<Row> stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.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 69% 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 index 0337f3597053..4e649d15b1ce 100644 --- 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 @@ -18,10 +18,10 @@ */ package org.apache.iceberg.flink.source; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Locale; import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -30,10 +30,12 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.avro.AvroSchemaUtil; @@ -43,68 +45,64 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.sink.AvroGenericRecordToRowDataMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.reader.AvroGenericRecordConverter; import org.apache.iceberg.flink.source.reader.AvroGenericRecordReaderFunction; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedGenericRecord { + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } - private final FileFormat fileFormat; - private final int parallelism; + @Parameter(index = 0) + private FileFormat fileFormat; - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } + @Parameter(index = 1) + private int parallelism; - @Test + @Parameter(index = 2) + private boolean useConverter; + + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { String dateStr = "2020-03-20"; Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); @@ -112,19 +110,19 @@ public void testPartitionedTable() throws Exception { expectedRecords.get(i).setField("dt", dateStr); } - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testProjection() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List<Record> expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); // select the "data" field (fieldId == 1) Schema projectedSchema = TypeUtil.select(TestFixtures.SCHEMA, Sets.newHashSet(1)); @@ -149,29 +147,20 @@ private List<Row> run( Configuration config = new Configuration(); config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { + try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { tableLoader.open(); table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder<GenericRecord> sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder<GenericRecord> sourceBuilder = - IcebergSource.<GenericRecord>builder() - .tableLoader(catalogResource.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -179,7 +168,6 @@ private List<Row> run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -200,4 +188,35 @@ private List<Row> run( return Lists.newArrayList(iter); } } + + private IcebergSource.Builder<GenericRecord> createSourceBuilderWithReaderFunction( + Table table, Schema projected, List<Expression> filters, Configuration config) { + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + projected, + null, + false, + table.io(), + table.encryption(), + filters); + + return IcebergSource.<GenericRecord>builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + private IcebergSource.Builder<GenericRecord> createSourceBuilderWithConverter( + Table table, Schema readSchema, Configuration config) { + AvroGenericRecordConverter converter = + AvroGenericRecordConverter.fromIcebergSchema(readSchema, table.name()); + return IcebergSource.forOutputType(converter) + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } } diff --git a/flink/v1.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 96% 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 index 0f41c5af4c95..d3713e296014 100644 --- 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 @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.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 62% 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 3f5af78704ca..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 @@ -21,9 +21,11 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.time.Duration; import java.util.Collection; import java.util.List; +import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import org.apache.flink.api.common.JobID; @@ -31,11 +33,14 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; @@ -43,43 +48,43 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceContinuous { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); + public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); + + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private final AtomicLong randomSeed = new AtomicLong(0L); @Test public void testTableScanThenIncremental() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -92,49 +97,53 @@ public void testTableScanThenIncremental() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @Test public void testTableScanThenIncrementalAfterExpiration() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot2 List<Record> batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - Assert.assertEquals(1, tableResource.table().history().size()); + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); ScanContext scanContext = ScanContext.builder() @@ -143,8 +152,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { @@ -152,33 +161,37 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List<Record> initialRecords = Lists.newArrayList(); initialRecords.addAll(batch1); initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @Test public void testEarliestSnapshot() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -193,41 +206,46 @@ public void testEarliestSnapshot() throws Exception { List<Row> result1 = waitForResult(iter, 4); List<Record> combinedBatch0AndBatch1 = Lists.newArrayList(batch0); combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @Test - public void testLatestSnapshot() throws Exception { + public void testLatestSnapshot(@InjectClusterClient ClusterClient<?> clusterClient) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -241,48 +259,52 @@ public void testLatestSnapshot() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // we want to make sure job is running first so that enumerator can // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + waitUntilJobIsRunning(clusterClient); // inclusive behavior for starting snapshot List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @Test public void testSpecificSnapshotId() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); ScanContext scanContext = ScanContext.builder() @@ -295,47 +317,51 @@ public void testSpecificSnapshotId() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @Test public void testSpecificSnapshotTimestamp() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List<Record> batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); // sleep for 2 ms to make sure snapshot1 has a higher timestamp value Thread.sleep(2); // snapshot1 List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); ScanContext scanContext = ScanContext.builder() @@ -349,25 +375,27 @@ public void testSpecificSnapshotTimestamp() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // consume data from snapshot1 List<Row> result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List<Record> batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List<Row> result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); } } @@ -375,27 +403,30 @@ public void testSpecificSnapshotTimestamp() throws Exception { public void testReadingFromBranch() throws Exception { String branch = "b1"; GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); List<Record> batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchBase); // create branch - tableResource + TABLE_EXTENSION .table() .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) .commit(); // snapshot1 to branch List<Record> batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch1); // snapshot2 to branch List<Record> batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch2); List<Record> branchExpectedRecords = Lists.newArrayList(); @@ -415,25 +446,26 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); // snapshot3 to branch List<Record> batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch3); List<Row> result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); // snapshot4 to branch List<Record> batch4 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch4); List<Row> result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); } // read only from main branch. Should contain only the first snapshot @@ -446,14 +478,14 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator<Row> iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List<Row> resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); List<Record> batchMain2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchMain2); resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); } } @@ -462,7 +494,7 @@ public void testValidation() { assertThatThrownBy( () -> IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(true) .endTag("tag") @@ -478,7 +510,7 @@ private DataStream<Row> createStream(ScanContext scanContext) throws Exception { DataStream<Row> stream = env.fromSource( IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(scanContext.isStreaming()) .streamingStartingStrategy(scanContext.streamingStartingStrategy()) @@ -490,7 +522,7 @@ private DataStream<Row> createStream(ScanContext scanContext) throws Exception { WatermarkStrategy.noWatermarks(), "icebergSource", TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); return stream; } @@ -520,4 +552,22 @@ public static List<JobID> 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<MetricGroup> 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 74% 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 index a378a0b93be3..938ae4d9bb0a 100644 --- 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 @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Path; import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,6 +31,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; @@ -41,7 +43,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -51,53 +56,67 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; - +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 120) public class TestIcebergSourceFailover { // Parallelism higher than 1, but lower than the number of splits used by some of our tests // The goal is to allow some splits to remain in the enumerator when restoring the state private static final int PARALLELISM = 2; private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } protected IcebergSource.Builder<RowData> sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) // Prevent combining splits .set( @@ -119,13 +138,13 @@ protected void assertRecords(Table table, List<Record> expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Disabled("Disabled for now as it is flaky on CI") @Test - public void testBoundedWithSavepoint() throws Exception { + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient<?> clusterClient) + throws Exception { List<Record> expectedRecords = Lists.newArrayList(); - Table sinkTable = sinkTableResource.table(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List<Record> records = generateRecords(2, i); expectedRecords.addAll(records); @@ -141,13 +160,8 @@ public void testBoundedWithSavepoint() throws Exception { // Write something, but do not finish before checkpoint is created RecordCounterToWait.waitForCondition(); CompletableFuture<String> savepoint = - miniClusterResource - .getClusterClient() - .stopWithSavepoint( - jobId, - false, - TEMPORARY_FOLDER.newFolder().toPath().toString(), - SavepointFormatType.CANONICAL); + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint @@ -170,19 +184,21 @@ public void testBoundedWithSavepoint() throws Exception { @Test public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); } - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { List<Record> expectedRecords = Lists.newArrayList(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List<Record> records = generateRecords(2, i); expectedRecords.addAll(records); @@ -197,29 +213,27 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio JobID jobId = jobClient.getJobID(); RecordCounterToWait.waitForCondition(); - triggerFailover( - failoverType, - jobId, - RecordCounterToWait::continueProcessing, - miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } @Test public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); } - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); List<Record> expectedRecords = Lists.newArrayList(); List<Record> batch = generateRecords(2, 0); @@ -247,8 +261,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); @@ -260,13 +274,13 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep expectedRecords.addAll(records); dataAppender.appendToTable(records); if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, () -> {}, miniCluster); } } // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { @@ -286,8 +300,8 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); } @@ -295,6 +309,20 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ + private static void runTestWithNewMiniCluster(ThrowingConsumer<MiniCluster, Exception> testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + private enum FailoverType { NONE, TM, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 90% 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 index cde39ec98d61..4f61d2f7308a 100644 --- 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 @@ -40,6 +40,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -49,11 +50,24 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + @Override protected IcebergSource.Builder<RowData> sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) // Prevent combining splits diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List<Record> batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream<Row> dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector<Row> collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator<Row> iterator = collector.getOutput()) { + List<Row> result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} diff --git a/flink/v1.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 93% 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 index 75f0a785a8c5..548940a842ce 100644 --- 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 @@ -53,7 +53,12 @@ public class TestIcebergSourceSql extends TestSqlBase { public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.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 91% 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 8341c8074f21..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,9 +18,10 @@ */ 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; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -37,6 +38,7 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.InMemoryReporter; @@ -49,24 +51,24 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; @@ -74,25 +76,24 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int RECORD_NUM_FOR_2_SPLITS = 200; private static final ConcurrentMap<Long, Integer> WINDOWS = Maps.newConcurrentMap(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) .withHaLeadershipControl() .build()); - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); /** * This is an integration test for watermark handling and windowing. Integration testing the @@ -259,7 +260,7 @@ public void apply( * the readers continue reading. */ @Test - public void testThrottling() throws Exception { + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { GenericAppenderHelper dataAppender = appender(); // Generate records in advance @@ -310,8 +311,7 @@ public void testThrottling() throws Exception { try (CloseableIterator<RowData> resultIterator = stream.collectAsync()) { JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); // Insert the first data into the table dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); @@ -353,7 +353,7 @@ public void testThrottling() throws Exception { protected IcebergSource<RowData> source() { return IcebergSource.<RowData>builder() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) .splitSize(100L) @@ -377,7 +377,7 @@ protected Record generateRecord(int minutes, String str) { private Optional<Gauge<Long>> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() + return REPORTER.findMetrics(jobID, metricsName).values().stream() .map(m -> (Gauge<Long>) m) .filter(m -> m.getValue() == withValue) .findFirst(); @@ -389,7 +389,7 @@ private GenericAppenderHelper appender() { hadoopConf.set("write.parquet.page-size-bytes", "64"); hadoopConf.set("write.parquet.row-group-size-bytes", "64"); return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); } private static RowData row(long time, long count) { diff --git a/flink/v1.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..f0d083060c1d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,192 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.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.TaskInfo; +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.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static 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<Row> 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<Row> 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<Row, Row> { + @Override + public Row map(Row row) throws Exception { + // Simulate slow subtask 0 with attempt 0 + TaskInfo taskInfo = getRuntimeContext().getTaskInfo(); + if (taskInfo.getIndexOfThisSubtask() == 0 && taskInfo.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); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + + // 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 86% 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 index 25ecec23d216..ce9054ad49b6 100644 --- 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 @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowDelta; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; @@ -40,20 +47,19 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestProjectMetaColumn { - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Iterable<Object[]> parameters() { return Lists.newArrayList( new Object[] {FileFormat.PARQUET}, @@ -61,13 +67,9 @@ public static Iterable<Object[]> parameters() { new Object[] {FileFormat.AVRO}); } - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, @@ -89,7 +91,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); + assertThat(rowData).isInstanceOf(GenericRowData.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); @@ -97,20 +99,20 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); } - @Test + @TestTemplate public void testV1SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(1); } - @Test + @TestTemplate public void testV2SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(2); } - @Test + @TestTemplate public void testV2RemoveMetaColumn() throws Exception { // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); @@ -132,7 +134,7 @@ public void testV2RemoveMetaColumn() throws Exception { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); + assertThat(rowData).isInstanceOf(RowDataProjection.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/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 81% 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 index 317301260f66..b701419a7499 100644 --- 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 @@ -18,11 +18,12 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSourceUtil { @Test @@ -30,31 +31,31 @@ public void testInferedParallelism() throws IOException { Configuration configuration = new Configuration(); // Empty table, infer parallelism should be at least 1 int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits // num : 2 parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + assertThat(parallelism).isEqualTo(2); // 2 splits and limit is 1 , max infer parallelism is default 100, // which is greater than splits num and limit, the parallelism is the limit value : 1 parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : // 1 parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); } } diff --git a/flink/v1.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<List<Record>> 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<RowData> dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List<RowData> 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.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 000000000000..952255a52b7c --- /dev/null +++ b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests 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 9a3c16e3ac91..1297ec0365b2 100644 --- a/format/spec.md +++ b/format/spec.md @@ -44,6 +44,14 @@ The primary change in version 2 adds delete files to encode rows that are delete In addition to row-level deletes, version 2 makes some requirements stricter for writers. The full set of changes are listed in [Appendix E](#version-2). +#### Version 3: Extended Types and Capabilities + +Version 3 of the Iceberg spec extends data types and existing metadata structures to add new capabilities: + +* New data types: nanosecond timestamp(tz) +* Default value support for columns +* Multi-argument transforms for partitioning and sorting + ## Goals @@ -113,9 +121,9 @@ Tables do not require random-access writes. Once written, data and metadata file Tables do not require rename, except for tables that use atomic rename to implement the commit operation for new metadata files. -## Specification +# Specification -#### Terms +### Terms * **Schema** -- Names and types of fields in a table. * **Partition spec** -- A definition of how partition values are derived from data fields. @@ -125,9 +133,9 @@ Tables do not require rename, except for tables that use atomic rename to implem * **Data file** -- A file that contains rows of a table. * **Delete file** -- A file that encodes rows of a table that are deleted by position or data values. -#### Writer requirements +### Writer requirements -Some tables in this spec have columns that specify requirements for v1 and v2 tables. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. +Some tables in this spec have columns that specify requirements for tables by version. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. | Requirement | Write behavior | |-------------|----------------| @@ -135,10 +143,10 @@ Some tables in this spec have columns that specify requirements for v1 and v2 ta | _optional_ | The field can be written or omitted | | _required_ | The field must be written | -Readers should be more permissive because v1 metadata files are allowed in v2 tables so that tables can be upgraded to v2 without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected v2 read behavior: +Readers should be more permissive because v1 metadata files are allowed in v2 tables (or later) so that tables can be upgraded to without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected read behavior for later versions: -| v1 | v2 | v2 read behavior | -|------------|------------|------------------| +| v1 | v2 | v2+ read behavior | +|------------|------------|-------------------| | | _optional_ | Read the field as _optional_ | | | _required_ | Read the field as _optional_; it may be missing in v1 files | | _optional_ | | Ignore the field | @@ -148,15 +156,21 @@ Readers should be more permissive because v1 metadata files are allowed in v2 ta | _required_ | _optional_ | Read the field as _optional_ | | _required_ | _required_ | Fill in a default or throw an exception if the field is missing | -Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. +Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required fields that were not present in or were optional in prior versions may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. + +### Writing data files + +All columns must be written to data files even if they introduce redundancy with metadata stored in manifest files (e.g. columns with identity partition transforms). Writing all columns provides a backup in case of corruption or bugs in the metadata layer. -### Schemas and Data Types +Writers are not allowed to commit files with a partition spec that contains a field with an unknown transform. + +## Schemas and Data Types A table's **schema** is a list of named columns. All data types are either primitives or nested types, which are maps, lists, or structs. A table schema is also a struct type. For the representations of these types in Avro, ORC, and Parquet file formats, see Appendix A. -#### Nested Types +### Nested Types A **`struct`** is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. Fields may have an optional comment or doc string. Fields can have [default values](#default-values). @@ -164,7 +178,7 @@ A **`list`** is a collection of values with some element type. The element field A **`map`** is a collection of key-value pairs with a key type and a value type. Both the key field and value field each have an integer id that is unique in the table schema. Map keys are required and map values can be either optional or required. Both map keys and map values may be any type, including nested types. -#### Primitive Types +### Primitive Types Supported primitive types are defined in the table below. Primitive types added after v1 have an "added by" version that is the first spec version in which the type is allowed. For example, nanosecond-precision timestamps are part of the v3 spec; using v3 types in v1 or v2 tables can break forward compatibility. @@ -175,12 +189,12 @@ Supported primitive types are defined in the table below. Primitive types added | | **`long`** | 64-bit signed integers | | | | **`float`** | [32-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | Can promote to double | | | **`double`** | [64-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | | -| | **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed [1], precision must be 38 or less | +| | **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed, precision must be 38 or less | | | **`date`** | Calendar date without timezone or time | | -| | **`time`** | Time of day without date, timezone | Microsecond precision [2] | -| | **`timestamp`** | Timestamp, microsecond precision, without timezone | [2] | +| | **`time`** | Time of day, microsecond precision, without date, timezone | | +| | **`timestamp`** | Timestamp, microsecond precision, without timezone | [1] | | | **`timestamptz`** | Timestamp, microsecond precision, with timezone | [2] | -| [v3](#version-3) | **`timestamp_ns`** | Timestamp, nanosecond precision, without timezone | [2] | +| [v3](#version-3) | **`timestamp_ns`** | Timestamp, nanosecond precision, without timezone | [1] | | [v3](#version-3) | **`timestamptz_ns`** | Timestamp, nanosecond precision, with timezone | [2] | | | **`string`** | Arbitrary-length character sequences | Encoded with UTF-8 [3] | | | **`uuid`** | Universally unique identifiers | Should use 16-byte fixed | @@ -189,16 +203,14 @@ Supported primitive types are defined in the table below. Primitive types added Notes: -1. Decimal scale is fixed and cannot be changed by schema evolution. Precision can only be widened. -2. `time`, `timestamp`, and `timestamptz` values are represented with _microsecond precision_. `timestamp_ns` and `timstamptz_ns` values are represented with _nanosecond precision_. - - Timestamp values _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). - - Timestamp values _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). +1. Timestamp values _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). +2. Timestamp values _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). 3. Character strings must be stored as UTF-8 encoded byte arrays. For details on how to serialize a schema to JSON, see Appendix C. -#### Default values +### Default values Default values can be tracked for struct fields (both nested structs and the top-level schema's struct). There can be two defaults with a field: @@ -212,7 +224,7 @@ The `initial-default` and `write-default` produce SQL default value behavior, wi Default values are attributes of fields in schemas and serialized with fields in the JSON format. See [Appendix C](#appendix-c-json-serialization). -#### Schema Evolution +### Schema Evolution Schemas may be evolved by type promotion or adding, deleting, renaming, or reordering fields in structs (both nested structs and the top-level schema’s struct). @@ -241,7 +253,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. @@ -264,7 +283,7 @@ Field mapping fields are constrained by the following rules: For details on serialization, see [Appendix C](#name-mapping-serialization). -#### Identifier Field IDs +### Identifier Field IDs A schema can optionally track the set of primitive fields that identify rows in a table, using the property `identifier-field-ids` (see JSON encoding in Appendix C). @@ -273,7 +292,7 @@ Two rows are the "same"---that is, the rows represent the same entity---if the i Identifier fields may be nested in structs but cannot be nested within maps or lists. Float, double, and optional fields cannot be used as identifier fields and a nested field cannot be used as an identifier field if it is nested in an optional struct, to avoid null values in identifiers. -#### Reserved Field IDs +### Reserved Field IDs Iceberg tables must not use field ids greater than 2147483447 (`Integer.MAX_VALUE - 200`). This id range is reserved for metadata columns that can be used in user data schemas, like the `_file` column that holds the file path in which a row was stored. @@ -291,7 +310,7 @@ The set of metadata columns is: | **`2147483544 row`** | `struct<...>` | Deleted row values, used in position-based delete files | -### Partitioning +## Partitioning Data files are stored in manifests with a tuple of partition values that are used in scans to filter out files that cannot contain records that match the scan’s filter predicate. Partition values for a data file must be the same for all records stored in the data file. (Manifests store data files from any partition, as long as the partition spec is the same for the data files.) @@ -306,11 +325,13 @@ The source columns, selected by ids, must be a primitive type and cannot be cont Partition specs capture the transform from table data to partition values. This is used to transform predicates to partition predicates, in addition to transforming data values. Deriving partition predicates from column predicates on the table data is used to separate the logical queries from physical storage: the partitioning can change and the correct partition filters are always derived from column predicates. This simplifies queries because users don’t have to supply both logical predicates and partition predicates. For more information, see Scan Planning below. -Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column ID, transform definition and partition name. Writers must not create a new parition spec if there already exists a compatible partition spec defined in the table. +Partition fields that use an unknown transform can be read by ignoring the partition field for the purpose of filtering data files during scan planning. In v1 and v2, readers should ignore fields with unknown transforms while reading; this behavior is required in v3. Writers are not allowed to commit data using a partition spec that contains a field with an unknown transform. + +Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column IDs, transform definition and partition name. Writers must not create a new parition spec if there already exists a compatible partition spec defined in the table. Partition field IDs must be reused if an existing partition spec contains an equivalent field. -#### Partition Transforms +### Partition Transforms | Transform name | Description | Source types | Result type | |-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------| @@ -328,7 +349,7 @@ All transforms must return `null` for a `null` input value. The `void` transform may be used to replace the transform in an existing partition field so that the field is effectively dropped in v1 tables. See partition evolution below. -#### Bucket Transform Details +### Bucket Transform Details Bucket partition transforms use a 32-bit hash of the source value. The 32-bit hash implementation is the 32-bit Murmur3 hash, x86 variant, seeded with 0. @@ -345,7 +366,7 @@ Notes: For hash function details by type, see Appendix B. -#### Truncate Transform Details +### Truncate Transform Details | **Type** | **Config** | **Truncate specification** | **Examples** | |---------------|-----------------------|------------------------------------------------------------------|----------------------------------| @@ -363,7 +384,7 @@ Notes: 4. In contrast to strings, binary values do not have an assumed encoding and are truncated to `L` bytes. -#### Partition Evolution +### Partition Evolution Table partitioning can be evolved by adding, removing, renaming, or reordering partition spec fields. @@ -380,7 +401,7 @@ In v1, partition field IDs were not tracked, but were assigned sequentially star 3. Only add partition fields at the end of the previous partition spec -### Sorting +## Sorting Users can sort their data within partitions by columns to gain performance. The information on how the data is sorted can be declared per data or delete file, by a **sort order**. @@ -400,7 +421,7 @@ Sorting floating-point numbers should produce the following behavior: `-NaN` < ` A data or delete file is associated with a sort order by the sort order's id within [a manifest](#manifests). Therefore, the table must declare all the sort orders for lookup. A table could also be configured with a default sort order id, indicating how the new data should be sorted by default. Writers should use this default sort order to sort the data on write, but are not required to if the default order is prohibitively expensive, as it would be for streaming writes. -### Manifests +## Manifests A manifest is an immutable Avro file that lists data files or delete files, along with each file’s partition data tuple, metrics, and tracking information. One or more manifest files are used to store a [snapshot](#snapshots), which tracks all of the files in a table at some point in time. Manifests are tracked by a [manifest list](#manifest-lists) for each table snapshot. @@ -468,7 +489,7 @@ The `partition` struct stores the tuple of partition values for each file. Its t The column metrics maps are used when filtering to select both data and delete files. For delete files, the metrics must store bounds and counts for all deleted rows, or must be omitted. Storing metrics for deleted rows ensures that the values can be used during job planning to find delete files that must be merged during a scan. -#### Manifest Entry Fields +### Manifest Entry Fields The manifest entry fields are used to keep track of the snapshot in which files were added or logically deleted. The `data_file` struct is nested inside of the manifest entry so that it can be easily passed to job planning without the manifest entry fields. @@ -486,7 +507,7 @@ Notes: 1. Technically, data files can be deleted when the last snapshot that contains the file as “live” data is garbage collected. But this is harder to detect and requires finding the diff of multiple snapshots. It is easier to track what files are deleted in a snapshot and delete them when that snapshot expires. It is not recommended to add a deleted file back to a table. Adding a deleted file can lead to edge cases where incremental deletes can break table snapshots. 2. Manifest list files are required in v2, so that the `sequence_number` and `snapshot_id` to inherit are always available. -#### Sequence Number Inheritance +### Sequence Number Inheritance Manifests track the sequence number when a data or delete file was added to the table. @@ -500,7 +521,7 @@ Inheriting sequence numbers through the metadata tree allows writing a new manif When reading v1 manifests with no sequence number column, sequence numbers for all files must default to 0. -### Snapshots +## Snapshots A snapshot consists of the following fields: @@ -533,7 +554,7 @@ Manifests for a snapshot are tracked by a manifest list. Valid snapshots are stored as a list in table metadata. For serialization, see Appendix C. -#### Manifest Lists +### Manifest Lists Snapshots are embedded in table metadata, but the list of manifests for a snapshot are stored in a separate manifest list file. @@ -577,7 +598,7 @@ Notes: 1. Lower and upper bounds are serialized to bytes using the single-object serialization in Appendix D. The type of used to encode the value is the type of the partition field data. 2. If -0.0 is a value of the partition field, the `lower_bound` must not be +0.0, and if +0.0 is a value of the partition field, the `upper_bound` must not be -0.0. -#### Scan Planning +### Scan Planning Scans are planned by reading the manifest files for the current snapshot. Deleted entries in data and delete manifests (those marked with status "DELETED") are not used in a scan. @@ -589,6 +610,8 @@ Scan predicates are converted to partition predicates using an _inclusive projec For example, an `events` table with a timestamp column named `ts` that is partitioned by `ts_day=day(ts)` is queried by users with ranges over the timestamp column: `ts > X`. The inclusive projection is `ts_day >= day(X)`, which is used to select files that may have matching rows. Note that, in most cases, timestamps just before `X` will be included in the scan because the file contains rows that match the predicate and rows that do not match the predicate. +The inclusive projection for an unknown partition transform is _true_ because the partition field is ignored and not used in filtering. + Scan predicates are also used to filter data and delete files using column bounds and counts that are stored by field id in manifests. The same filter logic can be used for both data and delete files because both store metrics of the rows either inserted or deleted. If metrics show that a delete file has no rows that match a scan predicate, it may be ignored just as a data file would be ignored [2]. Data files that match the query filter must be read by the scan. @@ -600,10 +623,10 @@ Delete files that match the query filter must be applied to data files at read t * A _position_ delete file must be applied to a data file when all of the following are true: - The data file's data sequence number is _less than or equal to_ the delete file's data sequence number - - The data file's partition (both spec and partition values) is equal to the delete file's partition + - The data file's partition (both spec and partition values) is equal [4] to the delete file's partition * An _equality_ delete file must be applied to a data file when all of the following are true: - The data file's data sequence number is _strictly less than_ the delete's data sequence number - - The data file's partition (both spec id and partition values) is equal to the delete file's partition _or_ the delete file's partition spec is unpartitioned + - The data file's partition (both spec id and partition values) is equal [4] to the delete file's partition _or_ the delete file's partition spec is unpartitioned In general, deletes are applied only to data files that are older and in the same partition, except for two special cases: @@ -616,8 +639,9 @@ Notes: 1. An alternative, *strict projection*, creates a partition predicate that will match a file if all of the rows in the file must match the scan predicate. These projections are used to calculate the residual predicates for each file in a scan. 2. For example, if `file_a` has rows with `id` between 1 and 10 and a delete file contains rows with `id` between 1 and 4, a scan for `id = 9` may ignore the delete file because none of the deletes can match a row that will be selected. 3. Floating point partition values are considered equal if their IEEE 754 floating-point "single format" bit layout are equal with NaNs normalized to have only the the most significant mantissa bit set (the equivelant of calling `Float.floatToIntBits` or `Double.doubleToLongBits` in Java). The Avro specification requires all floating point values to be encoded in this format. +4. Unknown partition transforms do not affect partition equality. Although partition fields with unknown transforms are ignored for filtering, the result of an unknown transform is still used when testing whether partition values are equal. -#### Snapshot Reference +### Snapshot References Iceberg tables keep track of branches and tags using snapshot references. Tags are labels for individual snapshots. Branches are mutable named references that can be updated by committing a new snapshot as the branch's referenced snapshot using the [Commit Conflict Resolution and Retry](#commit-conflict-resolution-and-retry) procedures. @@ -634,7 +658,7 @@ The snapshot reference object records all the information of a reference includi Valid snapshot references are stored as the values of the `refs` map in table metadata. For serialization, see Appendix C. -#### Snapshot Retention Policy +### Snapshot Retention Policy Table snapshots expire and are removed from metadata to allow removed or replaced data files to be physically deleted. The snapshot expiration procedure removes snapshots from table metadata and applies the table's retention policy. @@ -650,13 +674,13 @@ When expiring snapshots, retention policies in table and snapshot references are 2. The snapshot is not one of the first `min-snapshots-to-keep` in the branch (including the branch's referenced snapshot) 5. Expire any snapshot not in the set of snapshots to retain. -### Table Metadata +## Table Metadata Table metadata is stored as JSON. Each table metadata change creates a new table metadata file that is committed by an atomic operation. This operation is used to ensure that a new version of table metadata replaces the version on which it was based. This produces a linear history of table versions and ensures that concurrent writes are not lost. The atomic operation used to commit metadata depends on how tables are tracked and is not standardized by this spec. See the sections below for examples. -#### Table Metadata Fields +### Table Metadata Fields Table metadata consists of the following fields: @@ -688,7 +712,7 @@ Table metadata consists of the following fields: For serialization details, see Appendix C. -#### Table Statistics +### Table Statistics Table statistics files are valid [Puffin files](puffin-spec.md). Statistics are informational. A reader can choose to ignore statistics information. Statistics support is not required to read the table correctly. A table can contain @@ -716,7 +740,7 @@ Blob metadata is a struct with the following fields: | _optional_ | _optional_ | **`properties`** | `map<string, string>` | Additional properties associated with the statistic. Subset of Blob properties in the Puffin file. | -#### Partition Statistics +### Partition Statistics Partition statistics files are based on [partition statistics file spec](#partition-statistics-file). Partition statistics are not required for reading or planning and readers may ignore them. @@ -759,15 +783,15 @@ 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<field#1, field#2, field#3>. +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<field#1, field#2, field#3>`. -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<field#1, field#2>. +2) `spec#0` has two fields `{field#1, field#2}` +and then the table has evolved into `spec#1` which has just one field `{field#2}`. +The unified partition type looks like `Struct<field#1, field#2>`. -#### Commit Conflict Resolution and Retry +## Commit Conflict Resolution and Retry When two commits happen at the same time and are based on the same version, only one commit will succeed. In most cases, the failed commit can be applied to the new current version of table metadata and retried. Updates verify the conditions under which they can be applied to a new version and retry if those conditions are met. @@ -777,9 +801,11 @@ When two commits happen at the same time and are based on the same version, only * Table schema updates and partition spec changes must validate that the schema has not changed between the base version and the current version. -#### File System Tables +### File System Tables + +_Note: This file system based scheme to commit a metadata file is **deprecated** and will be removed in version 4 of this spec. The scheme is **unsafe** in object stores and local file systems._ -An atomic swap can be implemented using atomic rename in file systems that support it, like HDFS or most local file systems [1]. +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<V>.metadata.json`. To commit a new metadata version, `V+1`, the writer performs the following steps: @@ -794,7 +820,7 @@ Notes: 1. The file system table scheme is implemented in [HadoopTableOperations](../javadoc/{{ icebergVersion }}/index.html?org/apache/iceberg/hadoop/HadoopTableOperations.html). -#### Metastore Tables +### Metastore Tables The atomic swap needed to commit new versions of table metadata can be implemented by storing a pointer in a metastore or database that is updated with a check-and-put operation [1]. The check-and-put validates that the version of the table that a write is based on is still current and then makes the new metadata from the write the current version. @@ -811,7 +837,7 @@ Notes: 1. The metastore table scheme is partly implemented in [BaseMetastoreTableOperations](../javadoc/{{ icebergVersion }}/index.html?org/apache/iceberg/BaseMetastoreTableOperations.html). -### Delete Formats +## Delete Formats This section details how to encode row-level deletes in Iceberg delete files. Row-level deletes are not supported in v1. @@ -822,7 +848,7 @@ Row-level delete files are tracked by manifests, like data files. A separate set Both position and equality deletes allow encoding deleted row values with a delete. This can be used to reconstruct a stream of changes to a table. -#### Position Delete Files +### Position Delete Files Position-based delete files identify deleted rows by file and position in one or more data files, and may optionally contain the deleted row. @@ -847,11 +873,11 @@ The rows in the delete file must be sorted by `file_path` then `pos` to optimize * Sorting by `file_path` allows filter pushdown by file in columnar storage formats. * Sorting by `pos` allows filtering rows while scanning, to avoid keeping deletes in memory. -#### Equality Delete Files +### Equality Delete Files Equality delete files identify deleted rows in a collection of data files by one or more column values, and may optionally contain additional columns of the deleted row. -Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). Float and double columns cannot be used as delete columns in equality delete files. +Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). The column restrictions for columns used in equality delete files are the same as those for [identifier fields](#identifier-field-ids) with the exception that optional columns and columns nested under optional structs are allowed (if a parent struct column is null it implies the leaf column is null). A data row is deleted if its values are equal to all delete columns for any row in an equality delete file that applies to the row's data file (see [`Scan Planning`](#scan-planning)). @@ -899,7 +925,7 @@ equality_ids=[1, 2] If a delete column in an equality delete file is later dropped from the table, it must still be used when applying the equality deletes. If a column was added to a table and later used as a delete column in an equality delete file, the column value is read for older data files using normal projection rules (defaults to `null`). -#### Delete File Stats +### Delete File Stats Manifests hold the same statistics for delete files and data files. For delete files, the metrics describe the values that were deleted. @@ -1050,10 +1076,10 @@ The 32-bit hash implementation is 32-bit Murmur3 hash, x86 variant, seeded with | **`time`** | `hashLong(microsecsFromMidnight(v))` | `22:31:08` → `-662762989` | | **`timestamp`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-2047944441`<br />`2017-11-16T22:31:08.000001` → `-1207196810` | | **`timestamptz`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00` → `-2047944441`<br />`2017-11-16T14:31:08.000001-08:00` → `-1207196810` | -| **`timestamp_ns`** | `hashLong(nanosecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-737750069`<br />`2017-11-16T22:31:08.000001` → `-976603392`<br />`2017-11-16T22:31:08.000000001` → `-160215926` | -| **`timestamptz_ns`** | `hashLong(nanosecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00` → `-737750069`<br />`2017-11-16T14:31:08.000001-08:00` → `-976603392`<br />`2017-11-16T14:31:08.000000001-08:00` → `-160215926` | +| **`timestamp_ns`** | `hashLong(microsecsFromUnixEpoch(v))` [3] | `2017-11-16T22:31:08` → `-2047944441`<br />`2017-11-16T22:31:08.000001001` → `-1207196810` | +| **`timestamptz_ns`** | `hashLong(microsecsFromUnixEpoch(v))` [3]| `2017-11-16T14:31:08-08:00` → `-2047944441`<br />`2017-11-16T14:31:08.000001001-08:00` → `-1207196810` | | **`string`** | `hashBytes(utf8Bytes(v))` | `iceberg` → `1210000089` | -| **`uuid`** | `hashBytes(uuidBytes(v))` [3] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | +| **`uuid`** | `hashBytes(uuidBytes(v))` [4] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | | **`fixed(L)`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | | **`binary`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | @@ -1062,17 +1088,18 @@ The types below are not currently valid for bucketing, and so are not hashed. Ho | Primitive type | Hash specification | Test value | |--------------------|-------------------------------------------|--------------------------------------------| | **`boolean`** | `false: hashInt(0)`, `true: hashInt(1)` | `true` → `1392991556` | -| **`float`** | `hashLong(doubleToLongBits(double(v))` [4]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | -| **`double`** | `hashLong(doubleToLongBits(v))` [4]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | +| **`float`** | `hashLong(doubleToLongBits(double(v))` [5]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | +| **`double`** | `hashLong(doubleToLongBits(v))` [5]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | Notes: 1. Integer and long hash results must be identical for all integer values. This ensures that schema evolution does not change bucket partition values if integer types are promoted. 2. Decimal values are hashed using the minimum number of bytes required to hold the unscaled value as a two’s complement big-endian; this representation does not include padding bytes required for storage in a fixed-length array. Hash results are not dependent on decimal scale, which is part of the type, not the data value. -3. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: +3. Nanosecond timestamps must be converted to microsecond precision before hashing to ensure timestamps have the same hash value. +4. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: `F7 9C 3E 09 67 7C 4B BD A4 79 3F 34 9C B7 85 E7` -4. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. +5. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. ## Appendix C: JSON serialization @@ -1230,42 +1257,6 @@ Example ] } ] ``` -### Content File (Data and Delete) Serialization - -Content file (data or delete) is serialized as a JSON object according to the following table. - -| Metadata field |JSON representation|Example| -|--------------------------|--- |--- | -| **`spec-id`** |`JSON int`|`1`| -| **`content`** |`JSON string`|`DATA`, `POSITION_DELETES`, `EQUALITY_DELETES`| -| **`file-path`** |`JSON string`|`"s3://b/wh/data.db/table"`| -| **`file-format`** |`JSON string`|`AVRO`, `ORC`, `PARQUET`| -| **`partition`** |`JSON object: Partition data tuple using partition field ids for the struct field ids`|`{"1000":1}`| -| **`record-count`** |`JSON long`|`1`| -| **`file-size-in-bytes`** |`JSON long`|`1024`| -| **`column-sizes`** |`JSON object: Map from column id to the total size on disk of all regions that store the column.`|`{"keys":[3,4],"values":[100,200]}`| -| **`value-counts`** |`JSON object: Map from column id to number of values in the column (including null and NaN values)`|`{"keys":[3,4],"values":[90,180]}`| -| **`null-value-counts`** |`JSON object: Map from column id to number of null values in the column`|`{"keys":[3,4],"values":[10,20]}`| -| **`nan-value-counts`** |`JSON object: Map from column id to number of NaN values in the column`|`{"keys":[3,4],"values":[0,0]}`| -| **`lower-bounds`** |`JSON object: Map from column id to lower bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["01000000","02000000"]}`| -| **`upper-bounds`** |`JSON object: Map from column id to upper bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["05000000","0A000000"]}`| -| **`key-metadata`** |`JSON string: Encryption key metadata binary serialized as hexadecimal string`|`00000000000000000000000000000000`| -| **`split-offsets`** |`JSON list of long: Split offsets for the data file`|`[128,256]`| -| **`equality-ids`** |`JSON list of int: Field ids used to determine row equality in equality delete files`|`[1]`| -| **`sort-order-id`** |`JSON int`|`1`| - -### File Scan Task Serialization - -File scan task is serialized as a JSON object according to the following table. - -| Metadata field |JSON representation|Example| -|--------------------------|--- |--- | -| **`schema`** |`JSON object`|`See above, read schemas instead`| -| **`spec`** |`JSON object`|`See above, read partition specs instead`| -| **`data-file`** |`JSON object`|`See above, read content file instead`| -| **`delete-files`** |`JSON list of objects`|`See above, read content file instead`| -| **`residual-filter`** |`JSON object: residual filter expression`|`{"type":"eq","term":"id","value":1}`| - ## Appendix D: Single-value serialization ### Binary single-value serialization @@ -1331,7 +1322,7 @@ Default values are added to struct fields in v3. Types `timestamp_ns` and `timestamptz_ns` are added in v3. -All readers are required to read tables with unknown partition transforms, ignoring them. +All readers are required to read tables with unknown partition transforms, ignoring the unsupported partition fields when filtering. Writing v3 metadata: @@ -1419,3 +1410,14 @@ Writing v2 metadata: * `sort_columns` was removed Note that these requirements apply when writing data to a v2 table. Tables that are upgraded from v1 may contain metadata that does not follow these requirements. Implementations should remain backward-compatible with v1 metadata requirements. + +## Appendix F: Implementation Notes + +This section covers topics not required by the specification but recommendations for systems implementing the Iceberg specification to help maintain a uniform experience. + +### Point in Time Reads (Time Travel) + +Iceberg supports two types of histories for tables. A history of previous "current snapshots" stored in ["snapshot-log" table metadata](#table-metadata-fields) and [parent-child lineage stored in "snapshots"](#table-metadata-fields). These two histories +might indicate different snapshot IDs for a specific timestamp. The discrepancies can be caused by a variety of table operations (e.g. updating the `current-snapshot-id` can be used to set the snapshot of a table to any arbitrary snapshot, which might have a lineage derived from a table branch or no lineage at all). + +When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. diff --git a/gcp-bundle/build.gradle b/gcp-bundle/build.gradle index 98e67c214ce0..1b339a66c195 100644 --- a/gcp-bundle/build.gradle +++ b/gcp-bundle/build.gradle @@ -38,6 +38,10 @@ project(":iceberg-gcp-bundle") { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // relocate GCP-specific versions relocate 'com.fasterxml.jackson', 'org.apache.iceberg.gcp.shaded.com.fasterxml.jackson' relocate 'com.google.common', 'org.apache.iceberg.gcp.shaded.com.google.common' diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java index fea51d50cfd0..4483b0233669 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java @@ -186,7 +186,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java index bcd6099ac987..e52a1638827a 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java @@ -130,7 +130,7 @@ public void close() throws IOException { stream.close(); } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/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 5f07fb949278..8b1feda17109 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" -assertj-core = "3.26.0" -awaitility = "4.2.1" -awssdk-bom = "2.26.16" +avro = "1.12.0" +assertj-core = "3.26.3" +awaitility = "4.2.2" +awssdk-bom = "2.27.12" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" @@ -37,15 +37,15 @@ datasketches = "6.0.0" delta-standalone = "3.2.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.28.0" +errorprone-annotations = "2.31.0" findbugs-jsr305 = "3.0.2" -flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} -google-libraries-bom = "26.28.0" -guava = "33.2.1-jre" +flink120 = { strictly = "1.20.0"} +google-libraries-bom = "26.44.0" +guava = "33.3.0-jre" hadoop2 = "2.7.3" -hadoop3-client = "3.3.6" +hadoop3 = "3.3.6" httpcomponents-httpclient5 = "5.3.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" @@ -57,34 +57,36 @@ 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.23" junit = "5.10.1" -kafka = "3.7.1" +junit-platform = "1.10.3" +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.91.3" -netty-buffer = "4.1.111.Final" -netty-buffer-compat = "4.1.111.Final" +nessie = "0.95.0" +netty-buffer = "4.1.112.Final" +netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" -orc = "1.9.3" +orc = "1.9.4" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.2.0" -s3mock-junit5 = "2.11.0" +roaringbitmap = "1.2.1" +s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" -snowflake-jdbc = "3.16.1" +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.19.8" +spring-web = "5.3.39" +sqlite-jdbc = "3.46.1.0" +testcontainers = "1.20.1" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above @@ -107,12 +109,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 +121,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" } @@ -132,7 +134,8 @@ hadoop2-common = { module = "org.apache.hadoop:hadoop-common", version.ref = "ha hadoop2-hdfs = { module = "org.apache.hadoop:hadoop-hdfs", version.ref = "hadoop2" } hadoop2-mapreduce-client-core = { module = "org.apache.hadoop:hadoop-mapreduce-client-core", version.ref = "hadoop2" } hadoop2-minicluster = { module = "org.apache.hadoop:hadoop-minicluster", version.ref = "hadoop2" } -hadoop3-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop3-client" } +hadoop3-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop3" } +hadoop3-common = { module = "org.apache.hadoop:hadoop-common", version.ref = "hadoop3" } hive2-exec = { module = "org.apache.hive:hive-exec", version.ref = "hive2" } hive2-metastore = { module = "org.apache.hive:hive-metastore", version.ref = "hive2" } hive2-serde = { module = "org.apache.hive:hive-serde", version.ref = "hive2" } @@ -178,11 +181,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" } @@ -193,12 +191,20 @@ 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" } junit-jupiter-engine = { module = "org.junit.jupiter:junit-jupiter-engine", version.ref = "junit" } +junit-suite-api = { module = "org.junit.platform:junit-platform-suite-api", version.ref = "junit-platform" } +junit-suite-engine = { module = "org.junit.platform:junit-platform-suite-engine", version.ref = "junit-platform" } junit-vintage-engine = { module = "org.junit.vintage:junit-vintage-engine", version.ref = "junit" } kryo-shaded = { module = "com.esotericsoftware:kryo-shaded", version.ref = "kryo-shaded" } mockito-core = { module = "org.mockito:mockito-core", version.ref = "mockito" } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 381baa9cef1e..2b189974c29a 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,7 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionSha256Sum=544c35d6bd849ae8a5ed0bcea39ba677dc40f49df7d1835561582da2009b961d -distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip +distributionSha256Sum=5b9c5eb3f9fc2c94abaea57d90bd78747ca117ddbbf96c859d3741181a12bf2a +distributionUrl=https\://services.gradle.org/distributions/gradle-8.10-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 1757b45dfead..64084f24de2c 100755 --- a/gradlew +++ b/gradlew @@ -87,7 +87,7 @@ APP_BASE_NAME=${0##*/} APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit if [ ! -e $APP_HOME/gradle/wrapper/gradle-wrapper.jar ]; then - curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.7.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.10.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. @@ -207,7 +207,7 @@ fi DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' # Collect all arguments for the java command: -# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, +# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, # and any embedded shellness will be escaped. # * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be # treated as '${Hostname}' itself on the command line. diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index b4f49e29fc49..5c58222f0c01 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.hive; +import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Set; @@ -49,6 +50,7 @@ import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOTracker; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -79,6 +81,7 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa private ClientPool<IMetaStoreClient, TException> clients; private boolean listAllTables = false; private Map<String, String> catalogProperties; + private FileIOTracker fileIOTracker; public HiveCatalog() {} @@ -111,6 +114,7 @@ public void initialize(String inputName, Map<String, String> properties) { : CatalogUtil.loadFileIO(fileIOImpl, properties, conf); this.clients = new CachedClientPool(conf, properties); + this.fileIOTracker = new FileIOTracker(); } @Override @@ -512,7 +516,10 @@ private boolean isValidateNamespace(Namespace namespace) { public TableOperations newTableOps(TableIdentifier tableIdentifier) { String dbName = tableIdentifier.namespace().level(0); String tableName = tableIdentifier.name(); - return new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); + HiveTableOperations ops = + new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); + fileIOTracker.track(ops); + return ops; } @Override @@ -636,6 +643,14 @@ protected Map<String, String> properties() { return catalogProperties == null ? ImmutableMap.of() : catalogProperties; } + @Override + public void close() throws IOException { + super.close(); + if (fileIOTracker != null) { + fileIOTracker.close(); + } + } + @VisibleForTesting void setListAllTables(boolean listAllTables) { this.listAllTables = listAllTables; 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-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java index a1d4fb16b7e7..de6c8a0f6e5a 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java @@ -30,18 +30,18 @@ public enum HiveVersion { NOT_SUPPORTED(0); private final int order; - private static final HiveVersion current = calculate(); + private static final HiveVersion CURRENT = calculate(); HiveVersion(int order) { this.order = order; } public static HiveVersion current() { - return current; + return CURRENT; } public static boolean min(HiveVersion other) { - return current.order >= other.order; + return CURRENT.order >= other.order; } private static HiveVersion calculate() { diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java index 39cb1835c353..da28919ed647 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java @@ -55,17 +55,17 @@ public class HiveTableBaseTest { protected static HiveCatalog catalog; - static final Schema schema = + static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - static final Schema altered = + static final Schema ALTERED = new Schema( Types.StructType.of( required(1, "id", Types.LongType.get()), optional(2, "data", Types.LongType.get())) .fields()); - private static final PartitionSpec partitionSpec = builderFor(schema).identity("id").build(); + private static final PartitionSpec PARTITION_SPEC = builderFor(SCHEMA).identity("id").build(); private Path tableLocation; @@ -85,7 +85,7 @@ public static void initCatalog() { @BeforeEach public void createTestTable() { this.tableLocation = - new Path(catalog.createTable(TABLE_IDENTIFIER, schema, partitionSpec).location()); + new Path(catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PARTITION_SPEC).location()); } @AfterEach diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java index 6d8e9b4391c3..9ae3c97db47c 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableTest.java @@ -116,7 +116,7 @@ public void testCreate() throws TException { final Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Iceberg schema should match the loaded table - assertThat(icebergTable.schema().asStruct()).isEqualTo(schema.asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); } @Test @@ -172,7 +172,7 @@ public void testDropTable() throws IOException { Table table = catalog.loadTable(TABLE_IDENTIFIER); GenericRecordBuilder recordBuilder = - new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test")); + new GenericRecordBuilder(AvroSchemaUtil.convert(SCHEMA, "test")); List<GenericData.Record> records = Lists.newArrayList( recordBuilder.set("id", 1L).build(), @@ -181,7 +181,7 @@ public void testDropTable() throws IOException { String location1 = table.location().replace("file:", "") + "/data/file1.avro"; try (FileAppender<GenericData.Record> writer = - Avro.write(Files.localOutput(location1)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(location1)).schema(SCHEMA).named("test").build()) { for (GenericData.Record rec : records) { writer.add(rec); } @@ -189,7 +189,7 @@ public void testDropTable() throws IOException { String location2 = table.location().replace("file:", "") + "/data/file2.avro"; try (FileAppender<GenericData.Record> writer = - Avro.write(Files.localOutput(location2)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(location2)).schema(SCHEMA).named("test").build()) { for (GenericData.Record rec : records) { writer.add(rec); } @@ -257,14 +257,14 @@ public void testExistingTableUpdate() throws TException { // Only 2 snapshotFile Should exist and no manifests should exist assertThat(metadataVersionFiles(TABLE_NAME)).hasSize(2); assertThat(manifestFiles(TABLE_NAME)).hasSize(0); - assertThat(icebergTable.schema().asStruct()).isEqualTo(altered.asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(ALTERED.asStruct()); final org.apache.hadoop.hive.metastore.api.Table table = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); final List<String> hiveColumns = table.getSd().getCols().stream().map(FieldSchema::getName).collect(Collectors.toList()); final List<String> icebergColumns = - altered.columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); + ALTERED.columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); assertThat(hiveColumns).isEqualTo(icebergColumns); } @@ -378,7 +378,7 @@ public void testHiveTableAndIcebergTableWithSameName(TableType tableType) catalog.setListAllTables(false); // reset to default. // create an iceberg table with the same name - assertThatThrownBy(() -> catalog.createTable(identifier, schema, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> catalog.createTable(identifier, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(NoSuchIcebergTableException.class) .hasMessageStartingWith(String.format("Not an iceberg table: hive.%s", identifier)); @@ -444,7 +444,7 @@ public void testNonDefaultDatabaseLocation() throws IOException, TException { assertThat("file:" + nonDefaultLocation.getPath()).isEqualTo(namespaceMeta.get("location")); TableIdentifier tableIdentifier = TableIdentifier.of(namespace, TABLE_NAME); - catalog.createTable(tableIdentifier, schema); + catalog.createTable(tableIdentifier, SCHEMA); // Let's check the location loaded through the catalog Table table = catalog.loadTable(tableIdentifier); @@ -492,7 +492,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio TableIdentifier identifier = TableIdentifier.of(DB_NAME, "table1"); Table table = hadoopCatalog.createTable( - identifier, schema, PartitionSpec.unpartitioned(), Maps.newHashMap()); + identifier, SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap()); // insert some data String file1Location = appendData(table, "file1"); List<FileScanTask> tasks = Lists.newArrayList(table.newScan().planFiles()); @@ -534,7 +534,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio private String appendData(Table table, String fileName) throws IOException { GenericRecordBuilder recordBuilder = - new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test")); + new GenericRecordBuilder(AvroSchemaUtil.convert(SCHEMA, "test")); List<GenericData.Record> records = Lists.newArrayList( recordBuilder.set("id", 1L).build(), @@ -543,7 +543,7 @@ private String appendData(Table table, String fileName) throws IOException { String fileLocation = table.location().replace("file:", "") + "/data/" + fileName + ".avro"; try (FileAppender<GenericData.Record> writer = - Avro.write(Files.localOutput(fileLocation)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(fileLocation)).schema(SCHEMA).named("test").build()) { for (GenericData.Record rec : records) { writer.add(rec); } @@ -589,7 +589,7 @@ public void testEngineHiveEnabledDefault() throws TException { // Unset in hive-conf catalog.getConf().unset(ConfigProperties.ENGINE_HIVE_ENABLED); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); @@ -604,7 +604,7 @@ public void testEngineHiveEnabledConfig() throws TException { // Enable by hive-conf catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "true"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); @@ -615,7 +615,7 @@ public void testEngineHiveEnabledConfig() throws TException { // Disable by hive-conf catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "false"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned()); hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); assertHiveEnabled(hmsTable, false); @@ -631,7 +631,7 @@ public void testEngineHiveEnabledTableProperty() throws TException { tableProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "false"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned(), tableProperties); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned(), tableProperties); org.apache.hadoop.hive.metastore.api.Table hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); @@ -643,7 +643,7 @@ public void testEngineHiveEnabledTableProperty() throws TException { tableProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "false"); catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "true"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned(), tableProperties); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned(), tableProperties); hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); assertHiveEnabled(hmsTable, false); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index ccb8fc3a88f7..9249deb7598e 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -93,7 +93,7 @@ * Run all the tests from abstract of {@link CatalogTests} with few specific tests related to HIVE. */ public class TestHiveCatalog extends CatalogTests<HiveCatalog> { - private static final ImmutableMap meta = + private static final ImmutableMap META = ImmutableMap.of( "owner", "apache", "group", "iceberg", @@ -411,7 +411,7 @@ public void testCreateTableCustomSortOrder() throws Exception { @Test public void testDatabaseAndNamespaceWithLocation() throws Exception { Namespace namespace1 = Namespace.of("noLocation"); - catalog.createNamespace(namespace1, meta); + catalog.createNamespace(namespace1, META); Database database1 = HIVE_METASTORE_EXTENSION.metastoreClient().getDatabase(namespace1.toString()); @@ -430,7 +430,7 @@ public void testDatabaseAndNamespaceWithLocation() throws Exception { hiveLocalDir = hiveLocalDir.substring(0, hiveLocalDir.length() - 1); ImmutableMap newMeta = ImmutableMap.<String, String>builder() - .putAll(meta) + .putAll(META) .put("location", hiveLocalDir) .buildOrThrow(); Namespace namespace2 = Namespace.of("haveLocation"); @@ -527,12 +527,12 @@ private void createNamespaceAndVerifyOwnership( public void testLoadNamespaceMeta() throws TException { Namespace namespace = Namespace.of("dbname_load"); - catalog.createNamespace(namespace, meta); + catalog.createNamespace(namespace, META); Map<String, String> nameMata = catalog.loadNamespaceMetadata(namespace); assertThat(nameMata).containsEntry("owner", "apache"); assertThat(nameMata).containsEntry("group", "iceberg"); - assertThat(catalog.convertToDatabase(namespace, meta).getLocationUri()) + assertThat(catalog.convertToDatabase(namespace, META).getLocationUri()) .as("There no same location for db and namespace") .isEqualTo(nameMata.get("location")); } @@ -541,7 +541,7 @@ public void testLoadNamespaceMeta() throws TException { public void testNamespaceExists() throws TException { Namespace namespace = Namespace.of("dbname_exists"); - catalog.createNamespace(namespace, meta); + catalog.createNamespace(namespace, META); assertThat(catalog.namespaceExists(namespace)).as("Should true to namespace exist").isTrue(); assertThat(catalog.namespaceExists(Namespace.of("db2", "db2", "ns2"))) @@ -861,7 +861,7 @@ public void dropNamespace() { TableIdentifier identifier = TableIdentifier.of(namespace, "table"); Schema schema = getTestSchema(); - catalog.createNamespace(namespace, meta); + catalog.createNamespace(namespace, META); catalog.createTable(identifier, schema); Map<String, String> nameMata = catalog.loadNamespaceMetadata(namespace); assertThat(nameMata).containsEntry("owner", "apache"); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java index 61af2c7e79f3..d12a8503313b 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java @@ -86,7 +86,7 @@ public class TestHiveCommitLocks { private static HiveClientPool spyClientPool = null; private static CachedClientPool spyCachedClientPool = null; private static Configuration overriddenHiveConf; - private static final AtomicReference<IMetaStoreClient> spyClientRef = new AtomicReference<>(); + private static final AtomicReference<IMetaStoreClient> SPY_CLIENT_REF = new AtomicReference<>(); private static IMetaStoreClient spyClient = null; HiveTableOperations ops = null; TableMetadata metadataV1 = null; @@ -100,9 +100,9 @@ public class TestHiveCommitLocks { private static final String DB_NAME = "hivedb"; private static final String TABLE_NAME = "tbl"; - private static final Schema schema = + private static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - private static final PartitionSpec partitionSpec = builderFor(schema).identity("id").build(); + private static final PartitionSpec PARTITION_SPEC = builderFor(SCHEMA).identity("id").build(); static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME); @RegisterExtension @@ -143,8 +143,8 @@ public static void initCatalog() throws Exception { // cannot spy on RetryingHiveMetastoreClient as it is a proxy IMetaStoreClient client = spy(new HiveMetaStoreClient(HIVE_METASTORE_EXTENSION.hiveConf())); - spyClientRef.set(client); - return spyClientRef.get(); + SPY_CLIENT_REF.set(client); + return SPY_CLIENT_REF.get(); }); spyClientPool.run(IMetaStoreClient::isLocalMetaStore); // To ensure new client is created. @@ -153,15 +153,15 @@ public static void initCatalog() throws Exception { spy(new CachedClientPool(HIVE_METASTORE_EXTENSION.hiveConf(), Collections.emptyMap())); when(spyCachedClientPool.clientPool()).thenAnswer(invocation -> spyClientPool); - assertThat(spyClientRef.get()).isNotNull(); + assertThat(SPY_CLIENT_REF.get()).isNotNull(); - spyClient = spyClientRef.get(); + spyClient = SPY_CLIENT_REF.get(); } @BeforeEach public void before() throws Exception { this.tableLocation = - new Path(catalog.createTable(TABLE_IDENTIFIER, schema, partitionSpec).location()); + new Path(catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PARTITION_SPEC).location()); Table table = catalog.loadTable(TABLE_IDENTIFIER); ops = (HiveTableOperations) ((HasTableOperations) table).operations(); String dbName = TABLE_IDENTIFIER.namespace().level(0); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java index b3bbde460667..136c96934189 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommits.java @@ -302,7 +302,7 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted @Test public void testInvalidObjectException() { TableIdentifier badTi = TableIdentifier.of(DB_NAME, "`tbl`"); - assertThatThrownBy(() -> catalog.createTable(badTi, schema, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> catalog.createTable(badTi, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(ValidationException.class) .hasMessage(String.format("Invalid Hive object for %s.%s", DB_NAME, "`tbl`")); } @@ -310,7 +310,7 @@ public void testInvalidObjectException() { @Test public void testAlreadyExistsException() { assertThatThrownBy( - () -> catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned())) + () -> catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(AlreadyExistsException.class) .hasMessage(String.format("Table already exists: %s.%s", DB_NAME, TABLE_NAME)); } diff --git a/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/hive3-orc-bundle/build.gradle b/hive3-orc-bundle/build.gradle index 11bb2abf85c4..3ca89bd47435 100644 --- a/hive3-orc-bundle/build.gradle +++ b/hive3-orc-bundle/build.gradle @@ -50,6 +50,10 @@ project(':iceberg-hive3-orc-bundle') { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // Relocate dependencies to avoid conflicts relocate 'org.apache.orc.storage', 'org.apache.hadoop.hive' diff --git a/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java b/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java index 9ba99c8dec6c..3db2940f3cac 100644 --- a/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java +++ b/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java @@ -51,6 +51,7 @@ public LocalDateTime convert(Object o) { } @Override + @SuppressWarnings("JavaLocalDateTimeGetNano") public Timestamp getPrimitiveJavaObject(Object o) { if (o == null) { return null; diff --git a/jmh.gradle b/jmh.gradle index de50162cb099..a5d8d624270d 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (jdkVersion != '8' && jdkVersion != '11' && jdkVersion != '17') { - throw new GradleException("The JMH benchmarks must be run with JDK 8 or JDK 11 or JDK 17") +if (jdkVersion != '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 1fdd6bc6ea4c..785dc0a8c1ee 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -project(":iceberg-kafka-connect:iceberg-kafka-connect-events") { +project(':iceberg-kafka-connect:iceberg-kafka-connect-events') { dependencies { api project(':iceberg-api') implementation project(':iceberg-core') @@ -28,10 +28,10 @@ project(":iceberg-kafka-connect:iceberg-kafka-connect-events") { test { useJUnitPlatform() - } + } } -project(":iceberg-kafka-connect:iceberg-kafka-connect") { +project(':iceberg-kafka-connect:iceberg-kafka-connect') { dependencies { api project(':iceberg-api') implementation project(':iceberg-core') @@ -57,3 +57,187 @@ project(":iceberg-kafka-connect:iceberg-kafka-connect") { useJUnitPlatform() } } + +project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { + apply plugin: 'distribution' + + configurations { + hive { + extendsFrom runtimeClasspath + } + all { + exclude group: 'javax.activation', module: 'activation' + // force upgrades for dependencies with known vulnerabilities... + resolutionStrategy { + force 'org.codehaus.jettison:jettison:1.5.4' + force 'org.xerial.snappy:snappy-java:1.1.10.6' + force 'org.apache.commons:commons-compress:1.27.1' + force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' + } + } + } + + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + } + } + + configurations { + integrationImplementation.extendsFrom testImplementation + integrationRuntime.extendsFrom testRuntimeOnly + } + + dependencies { + implementation project(':iceberg-kafka-connect:iceberg-kafka-connect') + implementation(libs.hadoop3.common) { + exclude group: 'log4j' + exclude group: 'org.slf4j' + exclude group: 'ch.qos.reload4j' + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'com.fasterxml.woodstox' + exclude group: 'com.google.guava' + exclude group: 'com.google.protobuf' + exclude group: 'org.apache.curator' + exclude group: 'org.apache.zookeeper' + exclude group: 'org.apache.kerby' + exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + exclude group: 'org.apache.commons', module: 'commons-configuration2' + exclude group: 'org.apache.hadoop.thirdparty', module: 'hadoop-shaded-protobuf_3_7' + exclude group: 'org.codehaus.woodstox' + exclude group: 'org.eclipse.jetty' + } + implementation project(':iceberg-orc') + implementation project(':iceberg-parquet') + + implementation project(':iceberg-aws') + implementation platform(libs.awssdk.bom) + implementation 'software.amazon.awssdk:apache-client' + implementation 'software.amazon.awssdk:auth' + implementation "software.amazon.awssdk:http-auth-aws-crt" + implementation 'software.amazon.awssdk:iam' + implementation 'software.amazon.awssdk:sso' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:kms' + implementation 'software.amazon.awssdk:glue' + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:dynamodb' + implementation 'software.amazon.awssdk:lakeformation' + + implementation project(':iceberg-gcp') + implementation platform(libs.google.libraries.bom) + implementation 'com.google.cloud:google-cloud-storage' + + implementation project(':iceberg-azure') + implementation platform(libs.azuresdk.bom) + implementation 'com.azure:azure-storage-file-datalake' + implementation 'com.azure:azure-identity' + + hive project(':iceberg-hive-metastore') + hive(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-common' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-applicationhistoryservice' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-resourcemanager' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-web-proxy' + exclude group: 'org.apache.hive', module: 'hive-service-rpc' + exclude group: 'com.github.joshelser', module: 'dropwizard-metrics-hadoop-metrics2-reporter' + } + hive(libs.hadoop3.client) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + } + + integrationImplementation project(':iceberg-api') + integrationImplementation project(':iceberg-common') + integrationImplementation project(':iceberg-core') + integrationImplementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + integrationImplementation platform(libs.jackson.bom) + integrationImplementation libs.jackson.core + integrationImplementation libs.jackson.databind + integrationImplementation libs.jackson.databind + integrationImplementation libs.kafka.clients + integrationImplementation libs.kafka.connect.api + integrationImplementation libs.kafka.connect.json + integrationImplementation libs.testcontainers + integrationImplementation libs.httpcomponents.httpclient5 + integrationImplementation libs.awaitility + } + + task integrationTest(type: Test) { + useJUnitPlatform() + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + jvmArgs += project.property('extraJvmArgs') + } + + processResources { + filter { + it.replace('__VERSION__', project.version.toString()) + } + } + + distributions { + main { + contents { + from "${processResources.destinationDir}/manifest.json" + into('lib/') { + from configurations.runtimeClasspath + } + into('doc/') { + from "$rootDir/LICENSE" + } + into('assets/') { + from "${processResources.destinationDir}/iceberg.png" + } + } + } + hive { + contents { + from "${processResources.destinationDir}/manifest.json" + into('lib/') { + from configurations.hive + } + into('doc/') { + from "$rootDir/LICENSE" + } + into('assets/') { + from "${processResources.destinationDir}/iceberg.png" + } + } + } + } + + // there are no Maven artifacts so disable publishing tasks... + project.afterEvaluate { + project.tasks.matching { it.group == 'publishing' }.each {it.enabled = false} + } + + tasks.jar.enabled = false + + tasks.distTar.enabled = false + distZip.dependsOn processResources + installDist.dependsOn processResources + + tasks.hiveDistTar.enabled = false + hiveDistZip.dependsOn processResources + installHiveDist.dependsOn processResources + + integrationTest.dependsOn installDist + check.dependsOn integrationTest + + assemble.dependsOn distZip, hiveDistZip +} diff --git a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java index 50eaa1050485..f30eac892400 100644 --- a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java +++ b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -120,4 +121,23 @@ public Object get(int i) { throw new UnsupportedOperationException("Unknown field ordinal: " + i); } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableReference that = (TableReference) o; + return Objects.equals(catalog, that.catalog) + && Objects.equals(namespace, that.namespace) + && Objects.equals(name, that.name); + } + + @Override + public int hashCode() { + return Objects.hash(catalog, namespace, name); + } } diff --git a/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java b/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java index 8f1f7a601f86..48e268bf0561 100644 --- a/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java +++ b/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java @@ -44,8 +44,7 @@ private EventTestUtil() {} static final Schema SCHEMA = new Schema(ImmutableList.of(Types.NestedField.required(1, "id", Types.LongType.get()))); - static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("id").withSpecId(1).build(); + static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); static final SortOrder ORDER = SortOrder.builderFor(SCHEMA).sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST).build(); diff --git a/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml b/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml new file mode 100644 index 000000000000..202180289d96 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml @@ -0,0 +1,107 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +volumes: + data: {} + +services: + minio: + image: minio/minio + hostname: minio + environment: + - MINIO_ROOT_USER=minioadmin + - MINIO_ROOT_PASSWORD=minioadmin + ports: + - 9000:9000 + - 9001:9001 + volumes: + - data:/data + command: server /data --console-address ":9001" + + create-bucket: + image: minio/mc + depends_on: + - minio + volumes: + - data:/data + entrypoint: mc mb /data/bucket + + iceberg: + image: tabulario/iceberg-rest + depends_on: + - create-bucket + hostname: iceberg + ports: + - 8181:8181 + environment: + - AWS_REGION=us-east-1 + - CATALOG_WAREHOUSE=s3://bucket/warehouse/ + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9000 + - CATALOG_S3_PATH__STYLE__ACCESS=true + - CATALOG_S3_ACCESS__KEY__ID=minioadmin + - CATALOG_S3_SECRET__ACCESS__KEY=minioadmin + + kafka: + image: confluentinc/cp-kafka + hostname: kafka + ports: + - 29092:29092 + environment: + KAFKA_NODE_ID: 1 + KAFKA_LISTENERS: BROKER://0.0.0.0:9092,CONTROLLER://0.0.0.0:9093,EXTERNAL://0.0.0.0:29092 + KAFKA_ADVERTISED_LISTENERS: BROKER://kafka:9092,EXTERNAL://localhost:29092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: BROKER:PLAINTEXT,CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: BROKER + KAFKA_CONTROLLER_LISTENER_NAMES: CONTROLLER + KAFKA_CONTROLLER_QUORUM_VOTERS: 1@kafka:9093 + KAFKA_PROCESS_ROLES: broker,controller + KAFKA_JMX_PORT: 9101 + KAFKA_JMX_HOSTNAME: localhost + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_OFFSETS_TOPIC_NUM_PARTITIONS: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 + KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 + CLUSTER_ID: E-JXLvCaTiaUYDb1LwZ1JQ + + connect: + image: confluentinc/cp-kafka-connect + depends_on: + - kafka + hostname: connect + ports: + - 8083:8083 + volumes: + - ../build/install:/test/kafka-connect + environment: + CONNECT_REST_ADVERTISED_HOST_NAME: localhost + CONNECT_REST_PORT: 8083 + CONNECT_GROUP_ID: kc + CONNECT_CONFIG_STORAGE_TOPIC: kc-config + CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR: 1 + CONNECT_OFFSET_STORAGE_TOPIC: kc-offsets + CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR: 1 + CONNECT_STATUS_STORAGE_TOPIC: kc-storage + CONNECT_STATUS_STORAGE_REPLICATION_FACTOR: 1 + CONNECT_KEY_CONVERTER: org.apache.kafka.connect.json.JsonConverter + CONNECT_KEY_CONVERTER_SCHEMAS_ENABLE: false + CONNECT_VALUE_CONVERTER: org.apache.kafka.connect.json.JsonConverter + CONNECT_VALUE_CONVERTER_SCHEMAS_ENABLE: false + CONNECT_PLUGIN_PATH: /test/kafka-connect + CONNECT_BOOTSTRAP_SERVERS: kafka:9092 + CONNECT_OFFSET_FLUSH_INTERVAL_MS: 500 diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java new file mode 100644 index 000000000000..5c458ad3fa78 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.NullSource; +import org.junit.jupiter.params.provider.ValueSource; + +public class IntegrationDynamicTableTest extends IntegrationTestBase { + + private static final String TEST_DB = "test"; + private static final String TEST_TABLE1 = "tbl1"; + private static final String TEST_TABLE2 = "tbl2"; + private static final TableIdentifier TABLE_IDENTIFIER1 = TableIdentifier.of(TEST_DB, TEST_TABLE1); + private static final TableIdentifier TABLE_IDENTIFIER2 = TableIdentifier.of(TEST_DB, TEST_TABLE2); + + @BeforeEach + public void before() { + createTopic(testTopic(), TEST_TOPIC_PARTITIONS); + ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); + } + + @AfterEach + public void after() { + context().stopConnector(connectorName()); + deleteTopic(testTopic()); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE1)); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE2)); + ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSink(String branch) { + // partitioned table + catalog().createTable(TABLE_IDENTIFIER1, TestEvent.TEST_SCHEMA, TestEvent.TEST_SPEC); + // unpartitioned table + catalog().createTable(TABLE_IDENTIFIER2, TestEvent.TEST_SCHEMA); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema); + + List<DataFile> files = dataFiles(TABLE_IDENTIFIER1, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER1, branch); + + files = dataFiles(TABLE_IDENTIFIER2, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER2, branch); + } + + private void runTest(String branch, boolean useSchema) { + // set offset reset to earliest so we don't miss any test messages + KafkaConnectUtils.Config connectorConfig = + new KafkaConnectUtils.Config(connectorName()) + .config("topics", testTopic()) + .config("connector.class", IcebergSinkConnector.class.getName()) + .config("tasks.max", 2) + .config("consumer.override.auto.offset.reset", "earliest") + .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("key.converter.schemas.enable", false) + .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("value.converter.schemas.enable", useSchema) + .config("iceberg.tables.dynamic-enabled", true) + .config("iceberg.tables.route-field", "payload") + .config("iceberg.control.commit.interval-ms", 1000) + .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) + .config("iceberg.kafka.auto.offset.reset", "earliest"); + + context().connectorCatalogProperties().forEach(connectorConfig::config); + + if (branch != null) { + connectorConfig.config("iceberg.tables.default-commit-branch", branch); + } + + if (!useSchema) { + connectorConfig.config("value.converter.schemas.enable", false); + } + + context().startConnector(connectorConfig); + + TestEvent event1 = new TestEvent(1, "type1", Instant.now(), TEST_DB + "." + TEST_TABLE1); + TestEvent event2 = new TestEvent(2, "type2", Instant.now(), TEST_DB + "." + TEST_TABLE2); + TestEvent event3 = new TestEvent(3, "type3", Instant.now(), TEST_DB + ".tbl3"); + + send(testTopic(), event1, useSchema); + send(testTopic(), event2, useSchema); + send(testTopic(), event3, useSchema); + flush(); + + Awaitility.await() + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .untilAsserted(this::assertSnapshotAdded); + } + + private void assertSnapshotAdded() { + Table table = catalog().loadTable(TABLE_IDENTIFIER1); + assertThat(table.snapshots()).hasSize(1); + table = catalog().loadTable(TABLE_IDENTIFIER2); + assertThat(table.snapshots()).hasSize(1); + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java new file mode 100644 index 000000000000..7cffbd8838b2 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.NullSource; +import org.junit.jupiter.params.provider.ValueSource; + +public class IntegrationMultiTableTest extends IntegrationTestBase { + + private static final String TEST_DB = "test"; + private static final String TEST_TABLE1 = "foobar1"; + private static final String TEST_TABLE2 = "foobar2"; + private static final TableIdentifier TABLE_IDENTIFIER1 = TableIdentifier.of(TEST_DB, TEST_TABLE1); + private static final TableIdentifier TABLE_IDENTIFIER2 = TableIdentifier.of(TEST_DB, TEST_TABLE2); + + @BeforeEach + public void before() { + createTopic(testTopic(), TEST_TOPIC_PARTITIONS); + ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); + } + + @AfterEach + public void after() { + context().stopConnector(connectorName()); + deleteTopic(testTopic()); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE1)); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE2)); + ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSink(String branch) { + // partitioned table + catalog().createTable(TABLE_IDENTIFIER1, TestEvent.TEST_SCHEMA, TestEvent.TEST_SPEC); + // unpartitioned table + catalog().createTable(TABLE_IDENTIFIER2, TestEvent.TEST_SCHEMA); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema); + + List<DataFile> files = dataFiles(TABLE_IDENTIFIER1, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER1, branch); + + files = dataFiles(TABLE_IDENTIFIER2, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER2, branch); + } + + private void runTest(String branch, boolean useSchema) { + // set offset reset to earliest so we don't miss any test messages + KafkaConnectUtils.Config connectorConfig = + new KafkaConnectUtils.Config(connectorName()) + .config("topics", testTopic()) + .config("connector.class", IcebergSinkConnector.class.getName()) + .config("tasks.max", 2) + .config("consumer.override.auto.offset.reset", "earliest") + .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("key.converter.schemas.enable", false) + .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("value.converter.schemas.enable", useSchema) + .config( + "iceberg.tables", + String.format("%s.%s, %s.%s", TEST_DB, TEST_TABLE1, TEST_DB, TEST_TABLE2)) + .config("iceberg.tables.route-field", "type") + .config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE1), "type1") + .config(String.format("iceberg.table.%s.%s.route-regex", TEST_DB, TEST_TABLE2), "type2") + .config("iceberg.control.commit.interval-ms", 1000) + .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) + .config("iceberg.kafka.auto.offset.reset", "earliest"); + + context().connectorCatalogProperties().forEach(connectorConfig::config); + + if (branch != null) { + connectorConfig.config("iceberg.tables.default-commit-branch", branch); + } + + // use a schema for one of the cases + if (!useSchema) { + connectorConfig.config("value.converter.schemas.enable", false); + } + + context().startConnector(connectorConfig); + + TestEvent event1 = new TestEvent(1, "type1", Instant.now(), "hello world!"); + TestEvent event2 = new TestEvent(2, "type2", Instant.now(), "having fun?"); + TestEvent event3 = new TestEvent(3, "type3", Instant.now(), "ignore me"); + + send(testTopic(), event1, useSchema); + send(testTopic(), event2, useSchema); + send(testTopic(), event3, useSchema); + flush(); + + Awaitility.await() + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .untilAsserted(this::assertSnapshotAdded); + } + + private void assertSnapshotAdded() { + Table table = catalog().loadTable(TABLE_IDENTIFIER1); + assertThat(table.snapshots()).hasSize(1); + table = catalog().loadTable(TABLE_IDENTIFIER2); + assertThat(table.snapshots()).hasSize(1); + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java new file mode 100644 index 000000000000..80a74539311c --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +import java.time.Duration; +import java.time.Instant; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.TimestampType; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.NullSource; +import org.junit.jupiter.params.provider.ValueSource; + +public class IntegrationTest extends IntegrationTestBase { + + private static final String TEST_DB = "test"; + private static final String TEST_TABLE = "foobar"; + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(TEST_DB, TEST_TABLE); + + @BeforeEach + public void before() { + createTopic(testTopic(), TEST_TOPIC_PARTITIONS); + ((SupportsNamespaces) catalog()).createNamespace(Namespace.of(TEST_DB)); + } + + @AfterEach + public void after() { + context().stopConnector(connectorName()); + deleteTopic(testTopic()); + catalog().dropTable(TableIdentifier.of(TEST_DB, TEST_TABLE)); + ((SupportsNamespaces) catalog()).dropNamespace(Namespace.of(TEST_DB)); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSinkPartitionedTable(String branch) { + catalog().createTable(TABLE_IDENTIFIER, TestEvent.TEST_SCHEMA, TestEvent.TEST_SPEC); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema, ImmutableMap.of()); + + List<DataFile> files = dataFiles(TABLE_IDENTIFIER, branch); + // partition may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertThat(files.get(1).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSinkUnpartitionedTable(String branch) { + catalog().createTable(TABLE_IDENTIFIER, TestEvent.TEST_SCHEMA); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema, ImmutableMap.of()); + + List<DataFile> files = dataFiles(TABLE_IDENTIFIER, branch); + // may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.stream().mapToLong(DataFile::recordCount).sum()).isEqualTo(2); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSinkSchemaEvolution(String branch) { + Schema initialSchema = + new Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "type", Types.StringType.get()))); + catalog().createTable(TABLE_IDENTIFIER, initialSchema); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema, ImmutableMap.of("iceberg.tables.evolve-schema-enabled", "true")); + + List<DataFile> files = dataFiles(TABLE_IDENTIFIER, branch); + // may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.stream().mapToLong(DataFile::recordCount).sum()).isEqualTo(2); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + + // when not using a value schema, the ID data type will not be updated + Class<? extends Type> expectedIdType = + useSchema ? Types.LongType.class : Types.IntegerType.class; + + assertGeneratedSchema(useSchema, expectedIdType); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSinkAutoCreate(String branch) { + boolean useSchema = branch == null; // use a schema for one of the tests + + Map<String, String> extraConfig = Maps.newHashMap(); + extraConfig.put("iceberg.tables.auto-create-enabled", "true"); + if (useSchema) { + // partition the table for one of the tests + extraConfig.put("iceberg.tables.default-partition-by", "hour(ts)"); + } + + runTest(branch, useSchema, extraConfig); + + List<DataFile> files = dataFiles(TABLE_IDENTIFIER, branch); + // may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.stream().mapToLong(DataFile::recordCount).sum()).isEqualTo(2); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + + assertGeneratedSchema(useSchema, LongType.class); + + PartitionSpec spec = catalog().loadTable(TABLE_IDENTIFIER).spec(); + assertThat(spec.isPartitioned()).isEqualTo(useSchema); + } + + private void assertGeneratedSchema(boolean useSchema, Class<? extends Type> expectedIdType) { + Schema tableSchema = catalog().loadTable(TABLE_IDENTIFIER).schema(); + assertThat(tableSchema.findField("id").type()).isInstanceOf(expectedIdType); + assertThat(tableSchema.findField("type").type()).isInstanceOf(StringType.class); + assertThat(tableSchema.findField("payload").type()).isInstanceOf(StringType.class); + + if (!useSchema) { + // without a schema we can only map the primitive type + assertThat(tableSchema.findField("ts").type()).isInstanceOf(LongType.class); + // null values should be ignored when not using a value schema + assertThat(tableSchema.findField("op")).isNull(); + } else { + assertThat(tableSchema.findField("ts").type()).isInstanceOf(TimestampType.class); + assertThat(tableSchema.findField("op").type()).isInstanceOf(StringType.class); + } + } + + private void runTest(String branch, boolean useSchema, Map<String, String> extraConfig) { + // set offset reset to earliest so we don't miss any test messages + KafkaConnectUtils.Config connectorConfig = + new KafkaConnectUtils.Config(connectorName()) + .config("topics", testTopic()) + .config("connector.class", IcebergSinkConnector.class.getName()) + .config("tasks.max", 2) + .config("consumer.override.auto.offset.reset", "earliest") + .config("key.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("key.converter.schemas.enable", false) + .config("value.converter", "org.apache.kafka.connect.json.JsonConverter") + .config("value.converter.schemas.enable", useSchema) + .config("iceberg.tables", String.format("%s.%s", TEST_DB, TEST_TABLE)) + .config("iceberg.control.commit.interval-ms", 1000) + .config("iceberg.control.commit.timeout-ms", Integer.MAX_VALUE) + .config("iceberg.kafka.auto.offset.reset", "earliest"); + + context().connectorCatalogProperties().forEach(connectorConfig::config); + + if (branch != null) { + connectorConfig.config("iceberg.tables.default-commit-branch", branch); + } + + extraConfig.forEach(connectorConfig::config); + + context().startConnector(connectorConfig); + + TestEvent event1 = new TestEvent(1, "type1", Instant.now(), "hello world!"); + + Instant threeDaysAgo = Instant.now().minus(Duration.ofDays(3)); + TestEvent event2 = new TestEvent(2, "type2", threeDaysAgo, "having fun?"); + + send(testTopic(), event1, useSchema); + send(testTopic(), event2, useSchema); + flush(); + + Awaitility.await() + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofSeconds(1)) + .untilAsserted(this::assertSnapshotAdded); + } + + private void assertSnapshotAdded() { + try { + Table table = catalog().loadTable(TABLE_IDENTIFIER); + assertThat(table.snapshots()).hasSize(1); + } catch (NoSuchTableException e) { + fail("Table should exist"); + } + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java new file mode 100644 index 000000000000..f90d4da0379e --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.assertj.core.api.Condition; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +public class IntegrationTestBase { + + private final TestContext context = TestContext.INSTANCE; + private Catalog catalog; + private Admin admin; + private String connectorName; + private String testTopic; + + private KafkaProducer<String, String> producer; + + protected static final int TEST_TOPIC_PARTITIONS = 2; + + protected TestContext context() { + return context; + } + + protected Catalog catalog() { + return catalog; + } + + protected String connectorName() { + return connectorName; + } + + protected String testTopic() { + return testTopic; + } + + @BeforeEach + public void baseBefore() { + catalog = context.initLocalCatalog(); + producer = context.initLocalProducer(); + admin = context.initLocalAdmin(); + + this.connectorName = "test_connector-" + UUID.randomUUID(); + this.testTopic = "test-topic-" + UUID.randomUUID(); + } + + @AfterEach + public void baseAfter() { + try { + if (catalog instanceof AutoCloseable) { + ((AutoCloseable) catalog).close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + producer.close(); + admin.close(); + } + + protected void assertSnapshotProps(TableIdentifier tableIdentifier, String branch) { + Table table = catalog.loadTable(tableIdentifier); + Map<String, String> props = latestSnapshot(table, branch).summary(); + assertThat(props) + .hasKeySatisfying( + new Condition<String>() { + @Override + public boolean matches(String str) { + return str.startsWith("kafka.connect.offsets."); + } + }); + assertThat(props).containsKey("kafka.connect.commit-id"); + } + + protected List<DataFile> dataFiles(TableIdentifier tableIdentifier, String branch) { + Table table = catalog.loadTable(tableIdentifier); + return Lists.newArrayList(latestSnapshot(table, branch).addedDataFiles(table.io())); + } + + protected List<DeleteFile> deleteFiles(TableIdentifier tableIdentifier, String branch) { + Table table = catalog.loadTable(tableIdentifier); + return Lists.newArrayList(latestSnapshot(table, branch).addedDeleteFiles(table.io())); + } + + private Snapshot latestSnapshot(Table table, String branch) { + return branch == null ? table.currentSnapshot() : table.snapshot(branch); + } + + protected void createTopic(String topicName, int partitions) { + try { + admin + .createTopics(ImmutableList.of(new NewTopic(topicName, partitions, (short) 1))) + .all() + .get(10, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + protected void deleteTopic(String topicName) { + try { + admin.deleteTopics(ImmutableList.of(topicName)).all().get(10, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + protected void send(String topicName, TestEvent event, boolean useSchema) { + String eventStr = event.serialize(useSchema); + try { + producer.send(new ProducerRecord<>(topicName, Long.toString(event.id()), eventStr)).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + protected void flush() { + producer.flush(); + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java new file mode 100644 index 000000000000..098ab2395b34 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.hc.client5.http.classic.HttpClient; +import org.apache.hc.client5.http.classic.methods.HttpDelete; +import org.apache.hc.client5.http.classic.methods.HttpGet; +import org.apache.hc.client5.http.classic.methods.HttpPost; +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.http.io.entity.StringEntity; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.awaitility.Awaitility; + +public class KafkaConnectUtils { + + private static final HttpClient HTTP = HttpClients.createDefault(); + + // JavaBean-style for serialization + public static class Config { + + private final String name; + private final Map<String, Object> config = Maps.newHashMap(); + + public Config(String name) { + this.name = name; + } + + public String getName() { + return name; + } + + public Map<String, Object> getConfig() { + return config; + } + + public Config config(String key, Object value) { + config.put(key, value); + return this; + } + } + + public static void startConnector(Config config) { + try { + HttpPost request = + new HttpPost(String.format("http://localhost:%d/connectors", TestContext.CONNECT_PORT)); + String body = TestContext.MAPPER.writeValueAsString(config); + request.setHeader("Content-Type", "application/json"); + request.setEntity(new StringEntity(body)); + HTTP.execute(request, response -> null); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static void ensureConnectorRunning(String name) { + HttpGet request = + new HttpGet( + String.format( + "http://localhost:%d/connectors/%s/status", TestContext.CONNECT_PORT, name)); + Awaitility.await() + .atMost(60, TimeUnit.SECONDS) + .until( + () -> + HTTP.execute( + request, + response -> { + if (response.getCode() == HttpStatus.SC_OK) { + JsonNode root = + TestContext.MAPPER.readTree(response.getEntity().getContent()); + String connectorState = root.get("connector").get("state").asText(); + ArrayNode taskNodes = (ArrayNode) root.get("tasks"); + List<String> taskStates = Lists.newArrayList(); + taskNodes.forEach(node -> taskStates.add(node.get("state").asText())); + return "RUNNING".equals(connectorState) + && taskStates.stream().allMatch("RUNNING"::equals); + } + return false; + })); + } + + public static void stopConnector(String name) { + try { + HttpDelete request = + new HttpDelete( + String.format("http://localhost:%d/connectors/%s", TestContext.CONNECT_PORT, name)); + HTTP.execute(request, response -> null); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private KafkaConnectUtils() {} +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java new file mode 100644 index 000000000000..729d4bb264e5 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.File; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.StringSerializer; +import org.testcontainers.containers.ComposeContainer; +import org.testcontainers.containers.wait.strategy.Wait; + +public class TestContext { + + public static final TestContext INSTANCE = new TestContext(); + public static final ObjectMapper MAPPER = new ObjectMapper(); + public static final int CONNECT_PORT = 8083; + + private static final int MINIO_PORT = 9000; + private static final int CATALOG_PORT = 8181; + private static final String BOOTSTRAP_SERVERS = "localhost:29092"; + private static final String AWS_ACCESS_KEY = "minioadmin"; + private static final String AWS_SECRET_KEY = "minioadmin"; + private static final String AWS_REGION = "us-east-1"; + + private TestContext() { + ComposeContainer container = + new ComposeContainer(new File("./docker/docker-compose.yml")) + .waitingFor("connect", Wait.forHttp("/connectors")); + container.start(); + } + + public void startConnector(KafkaConnectUtils.Config config) { + KafkaConnectUtils.startConnector(config); + KafkaConnectUtils.ensureConnectorRunning(config.getName()); + } + + public void stopConnector(String name) { + KafkaConnectUtils.stopConnector(name); + } + + public Catalog initLocalCatalog() { + String localCatalogUri = "http://localhost:" + CATALOG_PORT; + RESTCatalog result = new RESTCatalog(); + result.initialize( + "local", + ImmutableMap.<String, String>builder() + .put(CatalogProperties.URI, localCatalogUri) + .put(CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.aws.s3.S3FileIO") + .put("s3.endpoint", "http://localhost:" + MINIO_PORT) + .put("s3.access-key-id", AWS_ACCESS_KEY) + .put("s3.secret-access-key", AWS_SECRET_KEY) + .put("s3.path-style-access", "true") + .put("client.region", AWS_REGION) + .build()); + return result; + } + + public Map<String, Object> connectorCatalogProperties() { + return ImmutableMap.<String, Object>builder() + .put( + "iceberg.catalog." + CatalogUtil.ICEBERG_CATALOG_TYPE, + CatalogUtil.ICEBERG_CATALOG_TYPE_REST) + .put("iceberg.catalog." + CatalogProperties.URI, "http://iceberg:" + CATALOG_PORT) + .put( + "iceberg.catalog." + CatalogProperties.FILE_IO_IMPL, + "org.apache.iceberg.aws.s3.S3FileIO") + .put("iceberg.catalog.s3.endpoint", "http://minio:" + MINIO_PORT) + .put("iceberg.catalog.s3.access-key-id", AWS_ACCESS_KEY) + .put("iceberg.catalog.s3.secret-access-key", AWS_SECRET_KEY) + .put("iceberg.catalog.s3.path-style-access", true) + .put("iceberg.catalog.client.region", AWS_REGION) + .build(); + } + + public KafkaProducer<String, String> initLocalProducer() { + return new KafkaProducer<>( + ImmutableMap.of( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, + BOOTSTRAP_SERVERS, + ProducerConfig.CLIENT_ID_CONFIG, + UUID.randomUUID().toString()), + new StringSerializer(), + new StringSerializer()); + } + + public Admin initLocalAdmin() { + return Admin.create( + ImmutableMap.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS)); + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.java new file mode 100644 index 000000000000..27de3885a4f9 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import java.time.Instant; +import java.util.Date; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.types.Types; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.storage.ConverterConfig; +import org.apache.kafka.connect.storage.ConverterType; + +public class TestEvent { + + public static final Schema TEST_SCHEMA = + new Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "type", Types.StringType.get()), + Types.NestedField.required(3, "ts", Types.TimestampType.withZone()), + Types.NestedField.required(4, "payload", Types.StringType.get())), + ImmutableSet.of(1)); + + public static final org.apache.kafka.connect.data.Schema TEST_CONNECT_SCHEMA = + SchemaBuilder.struct() + .field("id", org.apache.kafka.connect.data.Schema.INT64_SCHEMA) + .field("type", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) + .field("ts", Timestamp.SCHEMA) + .field("payload", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) + .field("op", org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA); + + public static final PartitionSpec TEST_SPEC = + PartitionSpec.builderFor(TEST_SCHEMA).day("ts").build(); + + private static final JsonConverter JSON_CONVERTER = new JsonConverter(); + + static { + JSON_CONVERTER.configure( + ImmutableMap.of(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName())); + } + + private final long id; + private final String type; + private final Instant ts; + private final String payload; + private final String op; + + public TestEvent(long id, String type, Instant ts, String payload) { + this(id, type, ts, payload, null); + } + + public TestEvent(long id, String type, Instant ts, String payload, String op) { + this.id = id; + this.type = type; + this.ts = ts; + this.payload = payload; + this.op = op; + } + + public long id() { + return id; + } + + protected String serialize(boolean useSchema) { + try { + Struct value = + new Struct(TEST_CONNECT_SCHEMA) + .put("id", id) + .put("type", type) + .put("ts", Date.from(ts)) + .put("payload", payload) + .put("op", op); + + String convertMethod = + useSchema ? "convertToJsonWithEnvelope" : "convertToJsonWithoutEnvelope"; + JsonNode json = + DynMethods.builder(convertMethod) + .hiddenImpl( + JsonConverter.class, org.apache.kafka.connect.data.Schema.class, Object.class) + .build(JSON_CONVERTER) + .invoke(TestEvent.TEST_CONNECT_SCHEMA, value); + return TestContext.MAPPER.writeValueAsString(json); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/main/resources/iceberg.png b/kafka-connect/kafka-connect-runtime/src/main/resources/iceberg.png new file mode 100644 index 000000000000..e4a99c3951e6 Binary files /dev/null and b/kafka-connect/kafka-connect-runtime/src/main/resources/iceberg.png differ diff --git a/kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json b/kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json new file mode 100644 index 000000000000..5b51e5dea875 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json @@ -0,0 +1,47 @@ +{ + "title": "Apache Iceberg Sink Connector", + "name": "iceberg-kafka-connect", + "logo": "assets/iceberg.png", + "version": "__VERSION__", + "description": "The Apache Iceberg Sink Connector for Kafka Connect is a sink connector for writing data from Kafka into Iceberg tables.", + + "component_types": [ + "sink" + ], + + "requirements": [], + + "features": { + "confluent_control_center_integration": true, + "delivery_guarantee": [ + "exactly_once" + ], + "kafka_connect_api": true, + "single_message_transforms": true, + "supported_encodings": [ + "any" + ] + }, + + "license": [ + { + "name": "Apache License, Version 2.0", + "url": "https://www.apache.org/licenses/LICENSE-2.0" + } + ], + + "owner": { + "name": "Apache Software Foundation", + "logo": "assets/iceberg.png", + "type": "organization", + "url": "https://iceberg.apache.org", + "username": "iceberg" + }, + + "support": { + "provider_name": "Iceberg OSS Community", + "logo": "assets/iceberg.png", + "summary": "Support for this connector is provided by the Iceberg open source community. You can open an issue in the Iceberg GitHub repo or post a question in the Iceberg Slack workspace in the #kafka-connect channel.", + "url": "https://github.com/apache/iceberg/issues" + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java new file mode 100644 index 000000000000..a3c6358e1bdf --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.common.DynClasses; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.common.DynMethods.BoundMethod; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class CatalogUtils { + + private static final Logger LOG = LoggerFactory.getLogger(CatalogUtils.class.getName()); + private static final List<String> HADOOP_CONF_FILES = + ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); + + static Catalog loadCatalog(IcebergSinkConfig config) { + return CatalogUtil.buildIcebergCatalog( + config.catalogName(), config.catalogProps(), loadHadoopConfig(config)); + } + + // use reflection here to avoid requiring Hadoop as a dependency + private static Object loadHadoopConfig(IcebergSinkConfig config) { + Class<?> configClass = + DynClasses.builder() + .impl("org.apache.hadoop.hdfs.HdfsConfiguration") + .impl("org.apache.hadoop.conf.Configuration") + .orNull() + .build(); + + if (configClass == null) { + LOG.info("Hadoop not found on classpath, not creating Hadoop config"); + return null; + } + + try { + Object result = DynConstructors.builder().hiddenImpl(configClass).build().newInstance(); + BoundMethod addResourceMethod = + DynMethods.builder("addResource").impl(configClass, URL.class).build(result); + BoundMethod setMethod = + DynMethods.builder("set").impl(configClass, String.class, String.class).build(result); + + // load any config files in the specified config directory + String hadoopConfDir = config.hadoopConfDir(); + if (hadoopConfDir != null) { + HADOOP_CONF_FILES.forEach( + confFile -> { + Path path = Paths.get(hadoopConfDir, confFile); + if (Files.exists(path)) { + try { + addResourceMethod.invoke(path.toUri().toURL()); + } catch (IOException e) { + LOG.warn("Error adding Hadoop resource {}, resource was not added", path, e); + } + } + }); + } + + // set any Hadoop properties specified in the sink config + config.hadoopProps().forEach(setMethod::invoke); + + LOG.info("Hadoop config initialized: {}", configClass.getName()); + return result; + } catch (Exception e) { + LOG.warn( + "Hadoop found on classpath but could not create config, proceeding without config", e); + } + return null; + } + + private CatalogUtils() {} +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java new file mode 100644 index 000000000000..edc217d1b0e4 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import java.util.Collection; +import org.apache.iceberg.catalog.Catalog; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +public interface Committer { + void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context); + + void stop(); + + void save(Collection<SinkRecord> sinkRecords); +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java new file mode 100644 index 000000000000..18ff118c7773 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import org.apache.iceberg.connect.channel.CommitterImpl; + +class CommitterFactory { + static Committer createCommitter(IcebergSinkConfig config) { + return new CommitterImpl(); + } + + private CommitterFactory() {} +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java index e64e183089cf..cf34b0bcd4c8 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConfig.java @@ -62,7 +62,7 @@ public class IcebergSinkConfig extends AbstractConfig { private static final String KAFKA_PROP_PREFIX = "iceberg.kafka."; private static final String TABLE_PROP_PREFIX = "iceberg.table."; private static final String AUTO_CREATE_PROP_PREFIX = "iceberg.tables.auto-create-props."; - private static final String WRITE_PROP_PREFIX = "iceberg.table.write-props."; + private static final String WRITE_PROP_PREFIX = "iceberg.tables.write-props."; private static final String CATALOG_NAME_PROP = "iceberg.catalog"; private static final String TABLES_PROP = "iceberg.tables"; @@ -80,7 +80,6 @@ public class IcebergSinkConfig extends AbstractConfig { private static final String TABLES_SCHEMA_CASE_INSENSITIVE_PROP = "iceberg.tables.schema-case-insensitive"; private static final String CONTROL_TOPIC_PROP = "iceberg.control.topic"; - private static final String CONTROL_GROUP_ID_PROP = "iceberg.control.group-id"; private static final String COMMIT_INTERVAL_MS_PROP = "iceberg.control.commit.interval-ms"; private static final int COMMIT_INTERVAL_MS_DEFAULT = 300_000; private static final String COMMIT_TIMEOUT_MS_PROP = "iceberg.control.commit.timeout-ms"; @@ -104,11 +103,7 @@ public class IcebergSinkConfig extends AbstractConfig { public static final ConfigDef CONFIG_DEF = newConfigDef(); public static String version() { - String kcVersion = IcebergSinkConfig.class.getPackage().getImplementationVersion(); - if (kcVersion == null) { - kcVersion = "unknown"; - } - return IcebergBuild.version() + "-kc-" + kcVersion; + return IcebergBuild.version(); } private static ConfigDef newConfigDef() { @@ -185,12 +180,6 @@ private static ConfigDef newConfigDef() { DEFAULT_CONTROL_TOPIC, Importance.MEDIUM, "Name of the control topic"); - configDef.define( - CONTROL_GROUP_ID_PROP, - ConfigDef.Type.STRING, - null, - Importance.MEDIUM, - "Name of the consumer group to store offsets"); configDef.define( CONNECT_GROUP_ID_PROP, ConfigDef.Type.STRING, @@ -370,16 +359,6 @@ public String controlTopic() { return getString(CONTROL_TOPIC_PROP); } - public String controlGroupId() { - String result = getString(CONTROL_GROUP_ID_PROP); - if (result != null) { - return result; - } - String connectorName = connectorName(); - Preconditions.checkNotNull(connectorName, "Connector name cannot be null"); - return DEFAULT_CONTROL_GROUP_PREFIX + connectorName; - } - public String connectGroupId() { String result = getString(CONNECT_GROUP_ID_PROP); if (result != null) { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java index 8be8518f4407..be1f9a50b8f6 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java @@ -44,9 +44,7 @@ public void start(Map<String, String> connectorProps) { @Override public Class<? extends Task> taskClass() { - // FIXME: update this when the connector channel is added - // return IcebergSinkTask.class; - return null; + return IcebergSinkTask.class; } @Override diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java new file mode 100644 index 000000000000..bb9370d3d563 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import java.util.Collection; +import java.util.Map; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergSinkTask extends SinkTask { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkTask.class); + + private IcebergSinkConfig config; + private Catalog catalog; + private Committer committer; + + @Override + public String version() { + return IcebergSinkConfig.version(); + } + + @Override + public void start(Map<String, String> props) { + this.config = new IcebergSinkConfig(props); + } + + @Override + public void open(Collection<TopicPartition> partitions) { + Preconditions.checkArgument(catalog == null, "Catalog already open"); + Preconditions.checkArgument(committer == null, "Committer already open"); + + catalog = CatalogUtils.loadCatalog(config); + committer = CommitterFactory.createCommitter(config); + committer.start(catalog, config, context); + } + + @Override + public void close(Collection<TopicPartition> partitions) { + close(); + } + + private void close() { + if (committer != null) { + committer.stop(); + committer = null; + } + + if (catalog != null) { + if (catalog instanceof AutoCloseable) { + try { + ((AutoCloseable) catalog).close(); + } catch (Exception e) { + LOG.warn("An error occurred closing catalog instance, ignoring...", e); + } + } + catalog = null; + } + } + + @Override + public void put(Collection<SinkRecord> sinkRecords) { + if (committer != null) { + committer.save(sinkRecords); + } + } + + @Override + public void flush(Map<TopicPartition, OffsetAndMetadata> currentOffsets) { + if (committer != null) { + committer.save(null); + } + } + + @Override + public Map<TopicPartition, OffsetAndMetadata> preCommit( + Map<TopicPartition, OffsetAndMetadata> currentOffsets) { + // offset commit is handled by the worker + return ImmutableMap.of(); + } + + @Override + public void stop() { + close(); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java new file mode 100644 index 000000000000..993fcf67c989 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.Offset; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class Channel { + + private static final Logger LOG = LoggerFactory.getLogger(Channel.class); + + private final String controlTopic; + private final String connectGroupId; + private final Producer<String, byte[]> producer; + private final Consumer<String, byte[]> consumer; + private final SinkTaskContext context; + private final Admin admin; + private final Map<Integer, Long> controlTopicOffsets = Maps.newHashMap(); + private final String producerId; + + Channel( + String name, + String consumerGroupId, + IcebergSinkConfig config, + KafkaClientFactory clientFactory, + SinkTaskContext context) { + this.controlTopic = config.controlTopic(); + this.connectGroupId = config.connectGroupId(); + this.context = context; + + String transactionalId = name + config.transactionalSuffix(); + this.producer = clientFactory.createProducer(transactionalId); + this.consumer = clientFactory.createConsumer(consumerGroupId); + this.admin = clientFactory.createAdmin(); + + this.producerId = UUID.randomUUID().toString(); + } + + protected void send(Event event) { + send(ImmutableList.of(event), ImmutableMap.of()); + } + + @SuppressWarnings("FutureReturnValueIgnored") + protected void send(List<Event> events, Map<TopicPartition, Offset> sourceOffsets) { + Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = Maps.newHashMap(); + sourceOffsets.forEach((k, v) -> offsetsToCommit.put(k, new OffsetAndMetadata(v.offset()))); + + List<ProducerRecord<String, byte[]>> recordList = + events.stream() + .map( + event -> { + LOG.info("Sending event of type: {}", event.type().name()); + byte[] data = AvroUtil.encode(event); + // key by producer ID to keep event order + return new ProducerRecord<>(controlTopic, producerId, data); + }) + .collect(Collectors.toList()); + + synchronized (producer) { + producer.beginTransaction(); + try { + // NOTE: we shouldn't call get() on the future in a transactional context, + // see docs for org.apache.kafka.clients.producer.KafkaProducer + recordList.forEach(producer::send); + if (!sourceOffsets.isEmpty()) { + producer.sendOffsetsToTransaction( + offsetsToCommit, KafkaUtils.consumerGroupMetadata(context)); + } + producer.commitTransaction(); + } catch (Exception e) { + try { + producer.abortTransaction(); + } catch (Exception ex) { + LOG.warn("Error aborting producer transaction", ex); + } + throw e; + } + } + } + + protected abstract boolean receive(Envelope envelope); + + protected void consumeAvailable(Duration pollDuration) { + ConsumerRecords<String, byte[]> records = consumer.poll(pollDuration); + while (!records.isEmpty()) { + records.forEach( + record -> { + // the consumer stores the offsets that corresponds to the next record to consume, + // so increment the record offset by one + controlTopicOffsets.put(record.partition(), record.offset() + 1); + + Event event = AvroUtil.decode(record.value()); + + if (event.groupId().equals(connectGroupId)) { + LOG.debug("Received event of type: {}", event.type().name()); + if (receive(new Envelope(event, record.partition(), record.offset()))) { + LOG.info("Handled event of type: {}", event.type().name()); + } + } + }); + records = consumer.poll(pollDuration); + } + } + + protected Map<Integer, Long> controlTopicOffsets() { + return controlTopicOffsets; + } + + protected void commitConsumerOffsets() { + Map<TopicPartition, OffsetAndMetadata> offsetsToCommit = Maps.newHashMap(); + controlTopicOffsets() + .forEach( + (k, v) -> + offsetsToCommit.put(new TopicPartition(controlTopic, k), new OffsetAndMetadata(v))); + consumer.commitSync(offsetsToCommit); + } + + void start() { + consumer.subscribe(ImmutableList.of(controlTopic)); + + // initial poll with longer duration so the consumer will initialize... + consumeAvailable(Duration.ofSeconds(1)); + } + + void stop() { + LOG.info("Channel stopping"); + producer.close(); + consumer.close(); + admin.close(); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java new file mode 100644 index 000000000000..6cad33c3e387 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.time.OffsetDateTime; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class CommitState { + private static final Logger LOG = LoggerFactory.getLogger(CommitState.class); + + private final List<Envelope> commitBuffer = Lists.newArrayList(); + private final List<DataComplete> readyBuffer = Lists.newArrayList(); + private long startTime; + private UUID currentCommitId; + private final IcebergSinkConfig config; + + CommitState(IcebergSinkConfig config) { + this.config = config; + } + + void addResponse(Envelope envelope) { + commitBuffer.add(envelope); + if (!isCommitInProgress()) { + DataWritten dataWritten = (DataWritten) envelope.event().payload(); + LOG.warn( + "Received commit response when no commit in progress, this can happen during recovery. Commit ID: {}", + dataWritten.commitId()); + } + } + + void addReady(Envelope envelope) { + DataComplete dataComplete = (DataComplete) envelope.event().payload(); + readyBuffer.add(dataComplete); + if (!isCommitInProgress()) { + LOG.warn( + "Received commit ready when no commit in progress, this can happen during recovery. Commit ID: {}", + dataComplete.commitId()); + } + } + + UUID currentCommitId() { + return currentCommitId; + } + + boolean isCommitInProgress() { + return currentCommitId != null; + } + + boolean isCommitIntervalReached() { + if (startTime == 0) { + startTime = System.currentTimeMillis(); + } + + return (!isCommitInProgress() + && System.currentTimeMillis() - startTime >= config.commitIntervalMs()); + } + + void startNewCommit() { + currentCommitId = UUID.randomUUID(); + startTime = System.currentTimeMillis(); + } + + void endCurrentCommit() { + readyBuffer.clear(); + currentCommitId = null; + } + + void clearResponses() { + commitBuffer.clear(); + } + + boolean isCommitTimedOut() { + if (!isCommitInProgress()) { + return false; + } + + if (System.currentTimeMillis() - startTime > config.commitTimeoutMs()) { + LOG.info("Commit timeout reached. Commit ID: {}", currentCommitId); + return true; + } + return false; + } + + boolean isCommitReady(int expectedPartitionCount) { + if (!isCommitInProgress()) { + return false; + } + + int receivedPartitionCount = + readyBuffer.stream() + .filter(payload -> payload.commitId().equals(currentCommitId)) + .mapToInt(payload -> payload.assignments().size()) + .sum(); + + if (receivedPartitionCount >= expectedPartitionCount) { + LOG.info( + "Commit {} ready, received responses for all {} partitions", + currentCommitId, + receivedPartitionCount); + return true; + } + + LOG.info( + "Commit {} not ready, received responses for {} of {} partitions, waiting for more", + currentCommitId, + receivedPartitionCount, + expectedPartitionCount); + + return false; + } + + Map<TableReference, List<Envelope>> tableCommitMap() { + return commitBuffer.stream() + .collect( + Collectors.groupingBy( + envelope -> ((DataWritten) envelope.event().payload()).tableReference())); + } + + OffsetDateTime validThroughTs(boolean partialCommit) { + boolean hasValidThroughTs = + !partialCommit + && readyBuffer.stream() + .flatMap(event -> event.assignments().stream()) + .allMatch(offset -> offset.timestamp() != null); + + OffsetDateTime result; + if (hasValidThroughTs) { + result = + readyBuffer.stream() + .flatMap(event -> event.assignments().stream()) + .map(TopicPartitionOffset::timestamp) + .min(Comparator.naturalOrder()) + .orElse(null); + } else { + result = null; + } + return result; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java new file mode 100644 index 000000000000..53b7b76e8ea0 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Set; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.connect.Committer; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CommitterImpl implements Committer { + + private static final Logger LOG = LoggerFactory.getLogger(CommitterImpl.class); + + private CoordinatorThread coordinatorThread; + private Worker worker; + + static class TopicPartitionComparator implements Comparator<TopicPartition> { + + @Override + public int compare(TopicPartition o1, TopicPartition o2) { + int result = o1.topic().compareTo(o2.topic()); + if (result == 0) { + result = Integer.compare(o1.partition(), o2.partition()); + } + return result; + } + } + + @Override + public void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) { + KafkaClientFactory clientFactory = new KafkaClientFactory(config.kafkaProps()); + + ConsumerGroupDescription groupDesc; + try (Admin admin = clientFactory.createAdmin()) { + groupDesc = KafkaUtils.consumerGroupDescription(config.connectGroupId(), admin); + } + + if (groupDesc.state() == ConsumerGroupState.STABLE) { + Collection<MemberDescription> members = groupDesc.members(); + Set<TopicPartition> partitions = context.assignment(); + if (isLeader(members, partitions)) { + LOG.info("Task elected leader, starting commit coordinator"); + Coordinator coordinator = new Coordinator(catalog, config, members, clientFactory, context); + coordinatorThread = new CoordinatorThread(coordinator); + coordinatorThread.start(); + } + } + + LOG.info("Starting commit worker"); + SinkWriter sinkWriter = new SinkWriter(catalog, config); + worker = new Worker(config, clientFactory, sinkWriter, context); + worker.start(); + } + + @Override + public void save(Collection<SinkRecord> sinkRecords) { + if (sinkRecords != null && !sinkRecords.isEmpty()) { + worker.save(sinkRecords); + } + processControlEvents(); + } + + @Override + public void stop() { + if (worker != null) { + worker.stop(); + worker = null; + } + + if (coordinatorThread != null) { + coordinatorThread.terminate(); + coordinatorThread = null; + } + } + + @VisibleForTesting + boolean isLeader(Collection<MemberDescription> members, Collection<TopicPartition> partitions) { + // there should only be one task assigned partition 0 of the first topic, + // so elect that one the leader + TopicPartition firstTopicPartition = + members.stream() + .flatMap(member -> member.assignment().topicPartitions().stream()) + .min(new TopicPartitionComparator()) + .orElseThrow( + () -> new ConnectException("No partitions assigned, cannot determine leader")); + + return partitions.contains(firstTopicPartition); + } + + private void processControlEvents() { + if (coordinatorThread != null && coordinatorThread.isTerminated()) { + throw new NotRunningException("Coordinator unexpectedly terminated"); + } + if (worker != null) { + worker.process(); + } + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java new file mode 100644 index 000000000000..7274f77e0c85 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.time.OffsetDateTime; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class Coordinator extends Channel { + + private static final Logger LOG = LoggerFactory.getLogger(Coordinator.class); + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; + private static final String VALID_THROUGH_TS_SNAPSHOT_PROP = "kafka.connect.valid-through-ts"; + private static final Duration POLL_DURATION = Duration.ofSeconds(1); + + private final Catalog catalog; + private final IcebergSinkConfig config; + private final int totalPartitionCount; + private final String snapshotOffsetsProp; + private final ExecutorService exec; + private final CommitState commitState; + + Coordinator( + Catalog catalog, + IcebergSinkConfig config, + Collection<MemberDescription> members, + KafkaClientFactory clientFactory, + SinkTaskContext context) { + // pass consumer group ID to which we commit low watermark offsets + super("coordinator", config.connectGroupId() + "-coord", config, clientFactory, context); + + this.catalog = catalog; + this.config = config; + this.totalPartitionCount = + members.stream().mapToInt(desc -> desc.assignment().topicPartitions().size()).sum(); + this.snapshotOffsetsProp = + String.format( + "kafka.connect.offsets.%s.%s", config.controlTopic(), config.connectGroupId()); + this.exec = ThreadPools.newWorkerPool("iceberg-committer", config.commitThreads()); + this.commitState = new CommitState(config); + } + + void process() { + if (commitState.isCommitIntervalReached()) { + // send out begin commit + commitState.startNewCommit(); + Event event = + new Event(config.connectGroupId(), new StartCommit(commitState.currentCommitId())); + send(event); + LOG.info("Commit {} initiated", commitState.currentCommitId()); + } + + consumeAvailable(POLL_DURATION); + + if (commitState.isCommitTimedOut()) { + commit(true); + } + } + + @Override + protected boolean receive(Envelope envelope) { + switch (envelope.event().payload().type()) { + case DATA_WRITTEN: + commitState.addResponse(envelope); + return true; + case DATA_COMPLETE: + commitState.addReady(envelope); + if (commitState.isCommitReady(totalPartitionCount)) { + commit(false); + } + return true; + } + return false; + } + + private void commit(boolean partialCommit) { + try { + doCommit(partialCommit); + } catch (Exception e) { + LOG.warn("Commit failed, will try again next cycle", e); + } finally { + commitState.endCurrentCommit(); + } + } + + private void doCommit(boolean partialCommit) { + Map<TableReference, List<Envelope>> commitMap = commitState.tableCommitMap(); + + String offsetsJson = offsetsJson(); + OffsetDateTime validThroughTs = commitState.validThroughTs(partialCommit); + + Tasks.foreach(commitMap.entrySet()) + .executeWith(exec) + .stopOnFailure() + .run( + entry -> { + commitToTable(entry.getKey(), entry.getValue(), offsetsJson, validThroughTs); + }); + + // we should only get here if all tables committed successfully... + commitConsumerOffsets(); + commitState.clearResponses(); + + Event event = + new Event( + config.connectGroupId(), + new CommitComplete(commitState.currentCommitId(), validThroughTs)); + send(event); + + LOG.info( + "Commit {} complete, committed to {} table(s), valid-through {}", + commitState.currentCommitId(), + commitMap.size(), + validThroughTs); + } + + private String offsetsJson() { + try { + return MAPPER.writeValueAsString(controlTopicOffsets()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private void commitToTable( + TableReference tableReference, + List<Envelope> envelopeList, + String offsetsJson, + OffsetDateTime validThroughTs) { + TableIdentifier tableIdentifier = tableReference.identifier(); + Table table; + try { + table = catalog.loadTable(tableIdentifier); + } catch (NoSuchTableException e) { + LOG.warn("Table not found, skipping commit: {}", tableIdentifier, e); + return; + } + + String branch = config.tableConfig(tableIdentifier.toString()).commitBranch(); + + Map<Integer, Long> committedOffsets = lastCommittedOffsetsForTable(table, branch); + + List<DataWritten> payloads = + envelopeList.stream() + .filter( + envelope -> { + Long minOffset = committedOffsets.get(envelope.partition()); + return minOffset == null || envelope.offset() >= minOffset; + }) + .map(envelope -> (DataWritten) envelope.event().payload()) + .collect(Collectors.toList()); + + List<DataFile> dataFiles = + payloads.stream() + .filter(payload -> payload.dataFiles() != null) + .flatMap(payload -> payload.dataFiles().stream()) + .filter(dataFile -> dataFile.recordCount() > 0) + .filter(distinctByKey(dataFile -> dataFile.path().toString())) + .collect(Collectors.toList()); + + List<DeleteFile> deleteFiles = + payloads.stream() + .filter(payload -> payload.deleteFiles() != null) + .flatMap(payload -> payload.deleteFiles().stream()) + .filter(deleteFile -> deleteFile.recordCount() > 0) + .filter(distinctByKey(deleteFile -> deleteFile.path().toString())) + .collect(Collectors.toList()); + + if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { + LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); + } else { + if (deleteFiles.isEmpty()) { + AppendFiles appendOp = table.newAppend(); + if (branch != null) { + appendOp.toBranch(branch); + } + appendOp.set(snapshotOffsetsProp, offsetsJson); + appendOp.set(COMMIT_ID_SNAPSHOT_PROP, commitState.currentCommitId().toString()); + if (validThroughTs != null) { + appendOp.set(VALID_THROUGH_TS_SNAPSHOT_PROP, validThroughTs.toString()); + } + dataFiles.forEach(appendOp::appendFile); + appendOp.commit(); + } else { + RowDelta deltaOp = table.newRowDelta(); + if (branch != null) { + deltaOp.toBranch(branch); + } + deltaOp.set(snapshotOffsetsProp, offsetsJson); + deltaOp.set(COMMIT_ID_SNAPSHOT_PROP, commitState.currentCommitId().toString()); + if (validThroughTs != null) { + deltaOp.set(VALID_THROUGH_TS_SNAPSHOT_PROP, validThroughTs.toString()); + } + dataFiles.forEach(deltaOp::addRows); + deleteFiles.forEach(deltaOp::addDeletes); + deltaOp.commit(); + } + + Long snapshotId = latestSnapshot(table, branch).snapshotId(); + Event event = + new Event( + config.connectGroupId(), + new CommitToTable( + commitState.currentCommitId(), tableReference, snapshotId, validThroughTs)); + send(event); + + LOG.info( + "Commit complete to table {}, snapshot {}, commit ID {}, valid-through {}", + tableIdentifier, + snapshotId, + commitState.currentCommitId(), + validThroughTs); + } + } + + private <T> Predicate<T> distinctByKey(Function<? super T, ?> keyExtractor) { + Map<Object, Boolean> seen = Maps.newConcurrentMap(); + return t -> seen.putIfAbsent(keyExtractor.apply(t), Boolean.TRUE) == null; + } + + private Snapshot latestSnapshot(Table table, String branch) { + if (branch == null) { + return table.currentSnapshot(); + } + return table.snapshot(branch); + } + + private Map<Integer, Long> lastCommittedOffsetsForTable(Table table, String branch) { + Snapshot snapshot = latestSnapshot(table, branch); + while (snapshot != null) { + Map<String, String> summary = snapshot.summary(); + String value = summary.get(snapshotOffsetsProp); + if (value != null) { + TypeReference<Map<Integer, Long>> typeRef = new TypeReference<Map<Integer, Long>>() {}; + try { + return MAPPER.readValue(value, typeRef); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + return ImmutableMap.of(); + } + + @Override + void stop() { + exec.shutdownNow(); + + // ensure coordinator tasks are shut down, else cause the sink worker to fail + try { + if (!exec.awaitTermination(1, TimeUnit.MINUTES)) { + throw new RuntimeException("Timed out waiting for coordinator shutdown"); + } + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted while waiting for coordinator shutdown", e); + } + + super.stop(); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java new file mode 100644 index 000000000000..6a31b17fc606 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class CoordinatorThread extends Thread { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorThread.class); + private static final String THREAD_NAME = "iceberg-coord"; + + private Coordinator coordinator; + private volatile boolean terminated; + + CoordinatorThread(Coordinator coordinator) { + super(THREAD_NAME); + this.coordinator = coordinator; + } + + @Override + public void run() { + try { + coordinator.start(); + } catch (Exception e) { + LOG.error("Coordinator error during start, exiting thread", e); + terminated = true; + } + + while (!terminated) { + try { + coordinator.process(); + } catch (Exception e) { + LOG.error("Coordinator error during process, exiting thread", e); + terminated = true; + } + } + + try { + coordinator.stop(); + } catch (Exception e) { + LOG.error("Coordinator error during stop, ignoring", e); + } + coordinator = null; + } + + boolean isTerminated() { + return terminated; + } + + void terminate() { + terminated = true; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java new file mode 100644 index 000000000000..87a93d058509 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import org.apache.iceberg.connect.events.Event; + +class Envelope { + private final Event event; + private final int partition; + private final long offset; + + Envelope(Event event, int partition, long offset) { + this.event = event; + this.partition = partition; + this.offset = offset; + } + + Event event() { + return event; + } + + int partition() { + return partition; + } + + long offset() { + return offset; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java new file mode 100644 index 000000000000..fd5d27ae34e2 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; + +class KafkaClientFactory { + private final Map<String, String> kafkaProps; + + KafkaClientFactory(Map<String, String> kafkaProps) { + this.kafkaProps = kafkaProps; + } + + Producer<String, byte[]> createProducer(String transactionalId) { + Map<String, Object> producerProps = Maps.newHashMap(kafkaProps); + producerProps.putIfAbsent(ProducerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString()); + producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId); + KafkaProducer<String, byte[]> result = + new KafkaProducer<>(producerProps, new StringSerializer(), new ByteArraySerializer()); + result.initTransactions(); + return result; + } + + Consumer<String, byte[]> createConsumer(String consumerGroupId) { + Map<String, Object> consumerProps = Maps.newHashMap(kafkaProps); + consumerProps.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + consumerProps.putIfAbsent(ConsumerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString()); + consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupId); + return new KafkaConsumer<>( + consumerProps, new StringDeserializer(), new ByteArrayDeserializer()); + } + + Admin createAdmin() { + Map<String, Object> adminProps = Maps.newHashMap(kafkaProps); + return Admin.create(adminProps); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java new file mode 100644 index 000000000000..be51fff8bfbc --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.util.concurrent.ExecutionException; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.DescribeConsumerGroupsResult; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkTaskContext; + +class KafkaUtils { + + private static final String CONTEXT_CLASS_NAME = + "org.apache.kafka.connect.runtime.WorkerSinkTaskContext"; + + static ConsumerGroupDescription consumerGroupDescription(String consumerGroupId, Admin admin) { + try { + DescribeConsumerGroupsResult result = + admin.describeConsumerGroups(ImmutableList.of(consumerGroupId)); + return result.describedGroups().get(consumerGroupId).get(); + + } catch (InterruptedException | ExecutionException e) { + throw new ConnectException( + "Cannot retrieve members for consumer group: " + consumerGroupId, e); + } + } + + static ConsumerGroupMetadata consumerGroupMetadata(SinkTaskContext context) { + return kafkaConsumer(context).groupMetadata(); + } + + @SuppressWarnings("unchecked") + private static Consumer<byte[], byte[]> kafkaConsumer(SinkTaskContext context) { + String contextClassName = context.getClass().getName(); + try { + return ((Consumer<byte[], byte[]>) + DynFields.builder().hiddenImpl(CONTEXT_CLASS_NAME, "consumer").build(context).get()); + } catch (Exception e) { + throw new ConnectException( + "Unable to retrieve consumer from context: " + contextClassName, e); + } + } + + private KafkaUtils() {} +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java new file mode 100644 index 000000000000..72a362ceacb0 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +public class NotRunningException extends RuntimeException { + public NotRunningException(String msg) { + super(msg); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java new file mode 100644 index 000000000000..7555b216cd45 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.Offset; +import org.apache.iceberg.connect.data.SinkWriter; +import org.apache.iceberg.connect.data.SinkWriterResult; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +class Worker extends Channel { + + private final IcebergSinkConfig config; + private final SinkTaskContext context; + private final SinkWriter sinkWriter; + + Worker( + IcebergSinkConfig config, + KafkaClientFactory clientFactory, + SinkWriter sinkWriter, + SinkTaskContext context) { + // pass transient consumer group ID to which we never commit offsets + super( + "worker", + IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(), + config, + clientFactory, + context); + + this.config = config; + this.context = context; + this.sinkWriter = sinkWriter; + } + + void process() { + consumeAvailable(Duration.ZERO); + } + + @Override + protected boolean receive(Envelope envelope) { + Event event = envelope.event(); + if (event.payload().type() != PayloadType.START_COMMIT) { + return false; + } + + SinkWriterResult results = sinkWriter.completeWrite(); + + // include all assigned topic partitions even if no messages were read + // from a partition, as the coordinator will use that to determine + // when all data for a commit has been received + List<TopicPartitionOffset> assignments = + context.assignment().stream() + .map( + tp -> { + Offset offset = results.sourceOffsets().get(tp); + if (offset == null) { + offset = Offset.NULL_OFFSET; + } + return new TopicPartitionOffset( + tp.topic(), tp.partition(), offset.offset(), offset.timestamp()); + }) + .collect(Collectors.toList()); + + UUID commitId = ((StartCommit) event.payload()).commitId(); + + List<Event> events = + results.writerResults().stream() + .map( + writeResult -> + new Event( + config.connectGroupId(), + new DataWritten( + writeResult.partitionStruct(), + commitId, + TableReference.of(config.catalogName(), writeResult.tableIdentifier()), + writeResult.dataFiles(), + writeResult.deleteFiles()))) + .collect(Collectors.toList()); + + Event readyEvent = new Event(config.connectGroupId(), new DataComplete(commitId, assignments)); + events.add(readyEvent); + + send(events, results.sourceOffsets()); + + return true; + } + + @Override + void stop() { + super.stop(); + sinkWriter.close(); + } + + void save(Collection<SinkRecord> sinkRecords) { + sinkWriter.save(sinkRecords); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java index 27ffc4de9973..6df6b091510b 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java @@ -32,16 +32,16 @@ import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; -public class IcebergWriter implements RecordWriter { +class IcebergWriter implements RecordWriter { private final Table table; private final String tableName; private final IcebergSinkConfig config; - private final List<WriterResult> writerResults; + private final List<IcebergWriterResult> writerResults; private RecordConverter recordConverter; private TaskWriter<Record> writer; - public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { + IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { this.table = table; this.tableName = tableName; this.config = config; @@ -50,7 +50,7 @@ public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { } private void initNewWriter() { - this.writer = Utilities.createTableWriter(table, tableName, config); + this.writer = RecordUtils.createTableWriter(table, tableName, config); this.recordConverter = new RecordConverter(table, config); } @@ -102,7 +102,7 @@ private void flush() { } writerResults.add( - new WriterResult( + new IcebergWriterResult( TableIdentifier.parse(tableName), Arrays.asList(writeResult.dataFiles()), Arrays.asList(writeResult.deleteFiles()), @@ -110,10 +110,10 @@ private void flush() { } @Override - public List<WriterResult> complete() { + public List<IcebergWriterResult> complete() { flush(); - List<WriterResult> result = Lists.newArrayList(writerResults); + List<IcebergWriterResult> result = Lists.newArrayList(writerResults); writerResults.clear(); return result; diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java index 47dcddcb9925..92f5af2d7a87 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java @@ -40,20 +40,19 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class IcebergWriterFactory { +class IcebergWriterFactory { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); private final Catalog catalog; private final IcebergSinkConfig config; - public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { + IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { this.catalog = catalog; this.config = config; } - public RecordWriter createWriter( - String tableName, SinkRecord sample, boolean ignoreMissingTable) { + RecordWriter createWriter(String tableName, SinkRecord sample, boolean ignoreMissingTable) { TableIdentifier identifier = TableIdentifier.parse(tableName); Table table; try { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/WriterResult.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java similarity index 96% rename from kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/WriterResult.java rename to kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java index cb3a700da247..58695a5572b5 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/WriterResult.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java @@ -24,14 +24,14 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types.StructType; -public class WriterResult { +public class IcebergWriterResult { private final TableIdentifier tableIdentifier; private final List<DataFile> dataFiles; private final List<DeleteFile> deleteFiles; private final StructType partitionStruct; - public WriterResult( + public IcebergWriterResult( TableIdentifier tableIdentifier, List<DataFile> dataFiles, List<DeleteFile> deleteFiles, diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java index 64ca44f03209..a7d2c90972d7 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java @@ -19,6 +19,7 @@ package org.apache.iceberg.connect.data; import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.kafka.connect.sink.SinkRecord; class NoOpWriter implements RecordWriter { @@ -28,9 +29,9 @@ public void write(SinkRecord record) { } @Override - public List<WriterResult> complete() { + public List<IcebergWriterResult> complete() { // NO-OP - return null; + return ImmutableList.of(); } @Override diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java new file mode 100644 index 000000000000..c4522a40711b --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import java.time.OffsetDateTime; +import java.util.Objects; + +public class Offset implements Comparable<Offset> { + + public static final Offset NULL_OFFSET = new Offset(null, null); + + private final Long offset; + private final OffsetDateTime timestamp; + + public Offset(Long offset, OffsetDateTime timestamp) { + this.offset = offset; + this.timestamp = timestamp; + } + + public Long offset() { + return offset; + } + + public OffsetDateTime timestamp() { + return timestamp; + } + + @Override + public int compareTo(Offset other) { + if (Objects.equals(this.offset, other.offset)) { + return 0; + } + if (this.offset == null || (other.offset != null && other.offset > this.offset)) { + return -1; + } + return 1; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java similarity index 68% rename from kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java rename to kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java index 4ff83f777527..5ac930739738 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java @@ -18,25 +18,14 @@ */ package org.apache.iceberg.connect.data; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.common.DynClasses; -import org.apache.iceberg.common.DynConstructors; -import org.apache.iceberg.common.DynMethods; -import org.apache.iceberg.common.DynMethods.BoundMethod; import org.apache.iceberg.connect.IcebergSinkConfig; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; @@ -46,7 +35,6 @@ import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Splitter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.primitives.Ints; @@ -55,71 +43,11 @@ import org.apache.iceberg.util.PropertyUtil; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public class Utilities { - - private static final Logger LOG = LoggerFactory.getLogger(Utilities.class.getName()); - private static final List<String> HADOOP_CONF_FILES = - ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); - - public static Catalog loadCatalog(IcebergSinkConfig config) { - return CatalogUtil.buildIcebergCatalog( - config.catalogName(), config.catalogProps(), loadHadoopConfig(config)); - } - - // use reflection here to avoid requiring Hadoop as a dependency - private static Object loadHadoopConfig(IcebergSinkConfig config) { - Class<?> configClass = - DynClasses.builder() - .impl("org.apache.hadoop.hdfs.HdfsConfiguration") - .impl("org.apache.hadoop.conf.Configuration") - .orNull() - .build(); - - if (configClass == null) { - LOG.info("Hadoop not found on classpath, not creating Hadoop config"); - return null; - } - - try { - Object result = DynConstructors.builder().hiddenImpl(configClass).build().newInstance(); - BoundMethod addResourceMethod = - DynMethods.builder("addResource").impl(configClass, URL.class).build(result); - BoundMethod setMethod = - DynMethods.builder("set").impl(configClass, String.class, String.class).build(result); - - // load any config files in the specified config directory - String hadoopConfDir = config.hadoopConfDir(); - if (hadoopConfDir != null) { - HADOOP_CONF_FILES.forEach( - confFile -> { - Path path = Paths.get(hadoopConfDir, confFile); - if (Files.exists(path)) { - try { - addResourceMethod.invoke(path.toUri().toURL()); - } catch (IOException e) { - LOG.warn("Error adding Hadoop resource {}, resource was not added", path, e); - } - } - }); - } - - // set any Hadoop properties specified in the sink config - config.hadoopProps().forEach(setMethod::invoke); - - LOG.info("Hadoop config initialized: {}", configClass.getName()); - return result; - } catch (Exception e) { - LOG.warn( - "Hadoop found on classpath but could not create config, proceeding without config", e); - } - return null; - } +class RecordUtils { @SuppressWarnings("unchecked") - public static Object extractFromRecordValue(Object recordValue, String fieldName) { + static Object extractFromRecordValue(Object recordValue, String fieldName) { List<String> fields = Splitter.on('.').splitToList(fieldName); if (recordValue instanceof Struct) { return valueFromStruct((Struct) recordValue, fields); @@ -243,5 +171,5 @@ public static TaskWriter<Record> createTableWriter( return writer; } - private Utilities() {} + private RecordUtils() {} } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java index 0b4d7566eab7..56438dde2e40 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java @@ -21,11 +21,11 @@ import java.util.List; import org.apache.kafka.connect.sink.SinkRecord; -public interface RecordWriter extends Cloneable { +interface RecordWriter extends Cloneable { void write(SinkRecord record); - List<WriterResult> complete(); + List<IcebergWriterResult> complete(); void close(); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java new file mode 100644 index 000000000000..f81155e13777 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkRecord; + +public class SinkWriter { + private final IcebergSinkConfig config; + private final IcebergWriterFactory writerFactory; + private final Map<String, RecordWriter> writers; + private final Map<TopicPartition, Offset> sourceOffsets; + + public SinkWriter(Catalog catalog, IcebergSinkConfig config) { + this.config = config; + this.writerFactory = new IcebergWriterFactory(catalog, config); + this.writers = Maps.newHashMap(); + this.sourceOffsets = Maps.newHashMap(); + } + + public void close() { + writers.values().forEach(RecordWriter::close); + } + + public SinkWriterResult completeWrite() { + List<IcebergWriterResult> writerResults = + writers.values().stream() + .flatMap(writer -> writer.complete().stream()) + .collect(Collectors.toList()); + Map<TopicPartition, Offset> offsets = Maps.newHashMap(sourceOffsets); + + writers.clear(); + sourceOffsets.clear(); + + return new SinkWriterResult(writerResults, offsets); + } + + public void save(Collection<SinkRecord> sinkRecords) { + sinkRecords.forEach(this::save); + } + + private void save(SinkRecord record) { + // the consumer stores the offsets that corresponds to the next record to consume, + // so increment the record offset by one + OffsetDateTime timestamp = + record.timestamp() == null + ? null + : OffsetDateTime.ofInstant(Instant.ofEpochMilli(record.timestamp()), ZoneOffset.UTC); + sourceOffsets.put( + new TopicPartition(record.topic(), record.kafkaPartition()), + new Offset(record.kafkaOffset() + 1, timestamp)); + + if (config.dynamicTablesEnabled()) { + routeRecordDynamically(record); + } else { + routeRecordStatically(record); + } + } + + private void routeRecordStatically(SinkRecord record) { + String routeField = config.tablesRouteField(); + + if (routeField == null) { + // route to all tables + config + .tables() + .forEach( + tableName -> { + writerForTable(tableName, record, false).write(record); + }); + + } else { + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + config + .tables() + .forEach( + tableName -> { + Pattern regex = config.tableConfig(tableName).routeRegex(); + if (regex != null && regex.matcher(routeValue).matches()) { + writerForTable(tableName, record, false).write(record); + } + }); + } + } + } + + private void routeRecordDynamically(SinkRecord record) { + String routeField = config.tablesRouteField(); + Preconditions.checkNotNull(routeField, "Route field cannot be null with dynamic routing"); + + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + String tableName = routeValue.toLowerCase(Locale.ROOT); + writerForTable(tableName, record, true).write(record); + } + } + + private String extractRouteValue(Object recordValue, String routeField) { + if (recordValue == null) { + return null; + } + Object routeValue = RecordUtils.extractFromRecordValue(recordValue, routeField); + return routeValue == null ? null : routeValue.toString(); + } + + private RecordWriter writerForTable( + String tableName, SinkRecord sample, boolean ignoreMissingTable) { + return writers.computeIfAbsent( + tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java new file mode 100644 index 000000000000..ef899102bb64 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.TopicPartition; + +public class SinkWriterResult { + private final List<IcebergWriterResult> writerResults; + private final Map<TopicPartition, Offset> sourceOffsets; + + public SinkWriterResult( + List<IcebergWriterResult> writerResults, Map<TopicPartition, Offset> sourceOffsets) { + this.writerResults = writerResults; + this.sourceOffsets = sourceOffsets; + } + + public List<IcebergWriterResult> writerResults() { + return writerResults; + } + + public Map<TopicPartition, Offset> sourceOffsets() { + return sourceOffsets; + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/UtilitiesTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/CatalogUtilsTest.java similarity index 60% rename from kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/UtilitiesTest.java rename to kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/CatalogUtilsTest.java index cfa1709da744..ce92b3efc3ed 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/UtilitiesTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/CatalogUtilsTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.connect.data; +package org.apache.iceberg.connect; import static org.assertj.core.api.Assertions.assertThat; @@ -27,19 +27,15 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.connect.IcebergSinkConfig; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.inmemory.InMemoryCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -public class UtilitiesTest { +public class CatalogUtilsTest { private static final String HADOOP_CONF_TEMPLATE = "<configuration><property><name>%s</name><value>%s</value></property></configuration>"; @@ -68,7 +64,7 @@ public void testLoadCatalogNoHadoopDir() { "iceberg.catalog.catalog-impl", TestCatalog.class.getName()); IcebergSinkConfig config = new IcebergSinkConfig(props); - Catalog result = Utilities.loadCatalog(config); + Catalog result = CatalogUtils.loadCatalog(config); assertThat(result).isInstanceOf(TestCatalog.class); @@ -102,7 +98,7 @@ public void testLoadCatalogWithHadoopDir(String confFile) throws IOException { "iceberg.catalog.catalog-impl", TestCatalog.class.getName()); IcebergSinkConfig config = new IcebergSinkConfig(props); - Catalog result = Utilities.loadCatalog(config); + Catalog result = CatalogUtils.loadCatalog(config); assertThat(result).isInstanceOf(TestCatalog.class); @@ -118,66 +114,4 @@ public void testLoadCatalogWithHadoopDir(String confFile) throws IOException { // check that core-site.xml was loaded assertThat(conf.get("foo")).isEqualTo("bar"); } - - @Test - public void testExtractFromRecordValueStruct() { - Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); - Struct val = new Struct(valSchema).put("key", 123L); - Object result = Utilities.extractFromRecordValue(val, "key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueStructNested() { - Schema idSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); - Schema dataSchema = SchemaBuilder.struct().field("id", idSchema).build(); - Schema valSchema = SchemaBuilder.struct().field("data", dataSchema).build(); - - Struct id = new Struct(idSchema).put("key", 123L); - Struct data = new Struct(dataSchema).put("id", id); - Struct val = new Struct(valSchema).put("data", data); - - Object result = Utilities.extractFromRecordValue(val, "data.id.key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueStructNull() { - Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); - Struct val = new Struct(valSchema).put("key", 123L); - - Object result = Utilities.extractFromRecordValue(val, ""); - assertThat(result).isNull(); - - result = Utilities.extractFromRecordValue(val, "xkey"); - assertThat(result).isNull(); - } - - @Test - public void testExtractFromRecordValueMap() { - Map<String, Object> val = ImmutableMap.of("key", 123L); - Object result = Utilities.extractFromRecordValue(val, "key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueMapNested() { - Map<String, Object> id = ImmutableMap.of("key", 123L); - Map<String, Object> data = ImmutableMap.of("id", id); - Map<String, Object> val = ImmutableMap.of("data", data); - - Object result = Utilities.extractFromRecordValue(val, "data.id.key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueMapNull() { - Map<String, Object> val = ImmutableMap.of("key", 123L); - - Object result = Utilities.extractFromRecordValue(val, ""); - assertThat(result).isNull(); - - result = Utilities.extractFromRecordValue(val, "xkey"); - assertThat(result).isNull(); - } } diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java new file mode 100644 index 000000000000..e6ffefbd9799 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.TableSinkConfig; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +public class ChannelTestBase { + protected static final String SRC_TOPIC_NAME = "src-topic"; + protected static final String CTL_TOPIC_NAME = "ctl-topic"; + protected static final String CONNECT_CONSUMER_GROUP_ID = "cg-connect"; + protected InMemoryCatalog catalog; + protected Table table; + protected IcebergSinkConfig config; + protected KafkaClientFactory clientFactory; + protected MockProducer<String, byte[]> producer; + protected MockConsumer<String, byte[]> consumer; + protected Admin admin; + + private InMemoryCatalog initInMemoryCatalog() { + InMemoryCatalog inMemoryCatalog = new InMemoryCatalog(); + inMemoryCatalog.initialize(null, ImmutableMap.of()); + return inMemoryCatalog; + } + + protected static final Namespace NAMESPACE = Namespace.of("db"); + protected static final String TABLE_NAME = "tbl"; + protected static final TableIdentifier TABLE_IDENTIFIER = + TableIdentifier.of(NAMESPACE, TABLE_NAME); + protected static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get())); + + protected static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; + protected static final String OFFSETS_SNAPSHOT_PROP = + String.format("kafka.connect.offsets.%s.%s", CTL_TOPIC_NAME, CONNECT_CONSUMER_GROUP_ID); + protected static final String VALID_THROUGH_TS_SNAPSHOT_PROP = "kafka.connect.valid-through-ts"; + + @BeforeEach + @SuppressWarnings("deprecation") + public void before() { + catalog = initInMemoryCatalog(); + catalog.createNamespace(NAMESPACE); + table = catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + + config = mock(IcebergSinkConfig.class); + when(config.controlTopic()).thenReturn(CTL_TOPIC_NAME); + when(config.commitThreads()).thenReturn(1); + when(config.connectGroupId()).thenReturn(CONNECT_CONSUMER_GROUP_ID); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + + TopicPartitionInfo partitionInfo = mock(TopicPartitionInfo.class); + when(partitionInfo.partition()).thenReturn(0); + TopicDescription topicDesc = + new TopicDescription(SRC_TOPIC_NAME, false, ImmutableList.of(partitionInfo)); + DescribeTopicsResult describeResult = mock(DescribeTopicsResult.class); + when(describeResult.values()) + .thenReturn(ImmutableMap.of(SRC_TOPIC_NAME, KafkaFuture.completedFuture(topicDesc))); + + admin = mock(Admin.class); + when(admin.describeTopics(anyCollection())).thenReturn(describeResult); + + producer = new MockProducer<>(false, new StringSerializer(), new ByteArraySerializer()); + producer.initTransactions(); + + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + + clientFactory = mock(KafkaClientFactory.class); + when(clientFactory.createProducer(any())).thenReturn(producer); + when(clientFactory.createConsumer(any())).thenReturn(consumer); + when(clientFactory.createAdmin()).thenReturn(admin); + } + + @AfterEach + public void after() throws IOException { + catalog.close(); + } + + protected void initConsumer() { + TopicPartition tp = new TopicPartition(CTL_TOPIC_NAME, 0); + consumer.rebalance(ImmutableList.of(tp)); + consumer.updateBeginningOffsets(ImmutableMap.of(tp, 0L)); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.java new file mode 100644 index 000000000000..a9fe1ad099cb --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.OffsetDateTime; +import java.util.UUID; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +public class CommitStateTest { + @Test + public void testIsCommitReady() { + TopicPartitionOffset tp = mock(TopicPartitionOffset.class); + + CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); + commitState.startNewCommit(); + + DataComplete payload1 = mock(DataComplete.class); + when(payload1.commitId()).thenReturn(commitState.currentCommitId()); + when(payload1.assignments()).thenReturn(ImmutableList.of(tp, tp)); + + DataComplete payload2 = mock(DataComplete.class); + when(payload2.commitId()).thenReturn(commitState.currentCommitId()); + when(payload2.assignments()).thenReturn(ImmutableList.of(tp)); + + DataComplete payload3 = mock(DataComplete.class); + when(payload3.commitId()).thenReturn(UUID.randomUUID()); + when(payload3.assignments()).thenReturn(ImmutableList.of(tp)); + + commitState.addReady(wrapInEnvelope(payload1)); + commitState.addReady(wrapInEnvelope(payload2)); + commitState.addReady(wrapInEnvelope(payload3)); + + assertThat(commitState.isCommitReady(3)).isTrue(); + assertThat(commitState.isCommitReady(4)).isFalse(); + } + + @Test + public void testGetValidThroughTs() { + DataComplete payload1 = mock(DataComplete.class); + TopicPartitionOffset tp1 = mock(TopicPartitionOffset.class); + OffsetDateTime ts1 = EventTestUtil.now(); + when(tp1.timestamp()).thenReturn(ts1); + + TopicPartitionOffset tp2 = mock(TopicPartitionOffset.class); + OffsetDateTime ts2 = ts1.plusSeconds(1); + when(tp2.timestamp()).thenReturn(ts2); + when(payload1.assignments()).thenReturn(ImmutableList.of(tp1, tp2)); + + DataComplete payload2 = mock(DataComplete.class); + TopicPartitionOffset tp3 = mock(TopicPartitionOffset.class); + OffsetDateTime ts3 = ts1.plusSeconds(2); + when(tp3.timestamp()).thenReturn(ts3); + when(payload2.assignments()).thenReturn(ImmutableList.of(tp3)); + + CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); + commitState.startNewCommit(); + + commitState.addReady(wrapInEnvelope(payload1)); + commitState.addReady(wrapInEnvelope(payload2)); + + assertThat(commitState.validThroughTs(false)).isEqualTo(ts1); + assertThat(commitState.validThroughTs(true)).isNull(); + + // null timestamp for one, so should not set a valid-through timestamp + DataComplete payload3 = mock(DataComplete.class); + TopicPartitionOffset tp4 = mock(TopicPartitionOffset.class); + when(tp4.timestamp()).thenReturn(null); + when(payload3.assignments()).thenReturn(ImmutableList.of(tp4)); + + commitState.addReady(wrapInEnvelope(payload3)); + + assertThat(commitState.validThroughTs(false)).isNull(); + assertThat(commitState.validThroughTs(true)).isNull(); + } + + private Envelope wrapInEnvelope(Payload payload) { + Event event = mock(Event.class); + when(event.payload()).thenReturn(payload); + return new Envelope(event, 0, 0); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.java new file mode 100644 index 000000000000..7c8ccf8ef669 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Optional; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.admin.MemberAssignment; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +public class CommitterImplTest { + + @Test + public void testIsLeader() { + CommitterImpl committer = new CommitterImpl(); + + MemberAssignment assignment1 = + new MemberAssignment( + ImmutableSet.of(new TopicPartition("topic1", 0), new TopicPartition("topic2", 1))); + MemberDescription member1 = + new MemberDescription(null, Optional.empty(), null, null, assignment1); + + MemberAssignment assignment2 = + new MemberAssignment( + ImmutableSet.of(new TopicPartition("topic2", 0), new TopicPartition("topic1", 1))); + MemberDescription member2 = + new MemberDescription(null, Optional.empty(), null, null, assignment2); + + List<MemberDescription> members = ImmutableList.of(member1, member2); + + List<TopicPartition> assignments = + ImmutableList.of(new TopicPartition("topic2", 1), new TopicPartition("topic1", 0)); + assertThat(committer.isLeader(members, assignments)).isTrue(); + + assignments = + ImmutableList.of(new TopicPartition("topic2", 0), new TopicPartition("topic1", 1)); + assertThat(committer.isLeader(members, assignments)).isFalse(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java new file mode 100644 index 000000000000..9c0b8122ae42 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.OffsetDateTime; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types.StructType; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class CoordinatorTest extends ChannelTestBase { + + @Test + public void testCommitAppend() { + Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); + + OffsetDateTime ts = EventTestUtil.now(); + UUID commitId = + coordinatorTest(ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), ts); + table.refresh(); + + assertThat(producer.history()).hasSize(3); + assertCommitTable(1, commitId, ts); + assertCommitComplete(2, commitId, ts); + + List<Snapshot> snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(1, snapshots.size()); + + Snapshot snapshot = snapshots.get(0); + Assertions.assertEquals(DataOperations.APPEND, snapshot.operation()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); + Assertions.assertEquals(0, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); + + Map<String, String> summary = snapshot.summary(); + Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); + Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); + Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + } + + @Test + public void testCommitDelta() { + OffsetDateTime ts = EventTestUtil.now(); + UUID commitId = + coordinatorTest( + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(EventTestUtil.createDeleteFile()), + ts); + + assertThat(producer.history()).hasSize(3); + assertCommitTable(1, commitId, ts); + assertCommitComplete(2, commitId, ts); + + List<Snapshot> snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(1, snapshots.size()); + + Snapshot snapshot = snapshots.get(0); + Assertions.assertEquals(DataOperations.OVERWRITE, snapshot.operation()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); + + Map<String, String> summary = snapshot.summary(); + Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); + Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); + Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + } + + @Test + public void testCommitNoFiles() { + OffsetDateTime ts = EventTestUtil.now(); + UUID commitId = coordinatorTest(ImmutableList.of(), ImmutableList.of(), ts); + + assertThat(producer.history()).hasSize(2); + assertCommitComplete(1, commitId, ts); + + List<Snapshot> snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(0, snapshots.size()); + } + + @Test + public void testCommitError() { + // this spec isn't registered with the table + PartitionSpec badPartitionSpec = + PartitionSpec.builderFor(SCHEMA).withSpecId(1).identity("id").build(); + DataFile badDataFile = + DataFiles.builder(badPartitionSpec) + .withPath(UUID.randomUUID() + ".parquet") + .withFormat(FileFormat.PARQUET) + .withFileSizeInBytes(100L) + .withRecordCount(5) + .build(); + + coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), null); + + // no commit messages sent + assertThat(producer.history()).hasSize(1); + + List<Snapshot> snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(0, snapshots.size()); + } + + private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { + byte[] bytes = producer.history().get(idx).value(); + Event commitTable = AvroUtil.decode(bytes); + assertThat(commitTable.type()).isEqualTo(PayloadType.COMMIT_TO_TABLE); + CommitToTable commitToTablePayload = (CommitToTable) commitTable.payload(); + assertThat(commitToTablePayload.commitId()).isEqualTo(commitId); + assertThat(commitToTablePayload.tableReference().identifier().toString()) + .isEqualTo(TABLE_IDENTIFIER.toString()); + assertThat(commitToTablePayload.validThroughTs()).isEqualTo(ts); + } + + private void assertCommitComplete(int idx, UUID commitId, OffsetDateTime ts) { + byte[] bytes = producer.history().get(idx).value(); + Event commitComplete = AvroUtil.decode(bytes); + assertThat(commitComplete.type()).isEqualTo(PayloadType.COMMIT_COMPLETE); + CommitComplete commitCompletePayload = (CommitComplete) commitComplete.payload(); + assertThat(commitCompletePayload.commitId()).isEqualTo(commitId); + assertThat(commitCompletePayload.validThroughTs()).isEqualTo(ts); + } + + private UUID coordinatorTest( + List<DataFile> dataFiles, List<DeleteFile> deleteFiles, OffsetDateTime ts) { + when(config.commitIntervalMs()).thenReturn(0); + when(config.commitTimeoutMs()).thenReturn(Integer.MAX_VALUE); + + SinkTaskContext context = mock(SinkTaskContext.class); + Coordinator coordinator = + new Coordinator(catalog, config, ImmutableList.of(), clientFactory, context); + coordinator.start(); + + // init consumer after subscribe() + initConsumer(); + + coordinator.process(); + + assertThat(producer.transactionCommitted()).isTrue(); + assertThat(producer.history()).hasSize(1); + + byte[] bytes = producer.history().get(0).value(); + Event commitRequest = AvroUtil.decode(bytes); + assertThat(commitRequest.type()).isEqualTo(PayloadType.START_COMMIT); + + UUID commitId = ((StartCommit) commitRequest.payload()).commitId(); + + Event commitResponse = + new Event( + config.connectGroupId(), + new DataWritten( + StructType.of(), + commitId, + new TableReference("catalog", ImmutableList.of("db"), "tbl"), + dataFiles, + deleteFiles)); + bytes = AvroUtil.encode(commitResponse); + consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 1, "key", bytes)); + + Event commitReady = + new Event( + config.connectGroupId(), + new DataComplete( + commitId, ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)))); + bytes = AvroUtil.encode(commitReady); + consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 2, "key", bytes)); + + when(config.commitIntervalMs()).thenReturn(0); + + coordinator.process(); + + return commitId; + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java new file mode 100644 index 000000000000..da0d881f8927 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.junit.jupiter.api.Test; + +public class CoordinatorThreadTest { + + @Test + public void testRun() { + Coordinator coordinator = mock(Coordinator.class); + CoordinatorThread coordinatorThread = new CoordinatorThread(coordinator); + + coordinatorThread.start(); + + verify(coordinator, timeout(1000)).start(); + verify(coordinator, timeout(1000).atLeast(1)).process(); + verify(coordinator, times(0)).stop(); + assertThat(coordinatorThread.isTerminated()).isFalse(); + + coordinatorThread.terminate(); + + verify(coordinator, timeout(1000)).stop(); + assertThat(coordinatorThread.isTerminated()).isTrue(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java new file mode 100644 index 000000000000..8c3625b74a5d --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import java.nio.ByteBuffer; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; + +class EventTestUtil { + private EventTestUtil() {} + + static final Schema SCHEMA = + new Schema(ImmutableList.of(Types.NestedField.required(1, "id", Types.LongType.get()))); + + static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + + static final SortOrder ORDER = + SortOrder.builderFor(SCHEMA).sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST).build(); + + static final Metrics METRICS = + new Metrics( + 1L, + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, ByteBuffer.wrap(new byte[10])), + ImmutableMap.of(1, ByteBuffer.wrap(new byte[10]))); + + static OffsetDateTime now() { + return OffsetDateTime.now(ZoneOffset.UTC).truncatedTo(ChronoUnit.MICROS); + } + + static DataFile createDataFile() { + PartitionData data = new PartitionData(SPEC.partitionType()); + data.set(0, 1L); + + return DataFiles.builder(SPEC) + .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[] {0})) + .withFileSizeInBytes(100L) + .withFormat(FileFormat.PARQUET) + .withMetrics(METRICS) + .withPartition(data) + .withPath("path/to/file.parquet") + .withSortOrder(ORDER) + .withSplitOffsets(ImmutableList.of(4L)) + .build(); + } + + static DeleteFile createDeleteFile() { + PartitionData data = new PartitionData(SPEC.partitionType()); + data.set(0, 1L); + + return FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1) + .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[] {0})) + .withFileSizeInBytes(100L) + .withFormat(FileFormat.PARQUET) + .withMetrics(METRICS) + .withPartition(data) + .withPath("path/to/file.parquet") + .withSortOrder(ORDER) + .withSplitOffsets(ImmutableList.of(4L)) + .build(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.java new file mode 100644 index 000000000000..577c28fe6375 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.data.IcebergWriterResult; +import org.apache.iceberg.connect.data.Offset; +import org.apache.iceberg.connect.data.SinkWriter; +import org.apache.iceberg.connect.data.SinkWriterResult; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.types.Types.StructType; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +public class WorkerTest extends ChannelTestBase { + + @Test + public void testSave() { + when(config.catalogName()).thenReturn("catalog"); + + try (MockedStatic<KafkaUtils> mockKafkaUtils = mockStatic(KafkaUtils.class)) { + ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + mockKafkaUtils + .when(() -> KafkaUtils.consumerGroupMetadata(any())) + .thenReturn(consumerGroupMetadata); + + SinkTaskContext context = mock(SinkTaskContext.class); + TopicPartition topicPartition = new TopicPartition(SRC_TOPIC_NAME, 0); + when(context.assignment()).thenReturn(ImmutableSet.of(topicPartition)); + + IcebergWriterResult writeResult = + new IcebergWriterResult( + TableIdentifier.parse(TABLE_NAME), + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(), + StructType.of()); + + Map<TopicPartition, Offset> offsets = + ImmutableMap.of(topicPartition, new Offset(1L, EventTestUtil.now())); + + SinkWriterResult sinkWriterResult = + new SinkWriterResult(ImmutableList.of(writeResult), offsets); + SinkWriter sinkWriter = mock(SinkWriter.class); + when(sinkWriter.completeWrite()).thenReturn(sinkWriterResult); + + Worker worker = new Worker(config, clientFactory, sinkWriter, context); + worker.start(); + + // init consumer after subscribe() + initConsumer(); + + // save a record + Map<String, Object> value = ImmutableMap.of(); + SinkRecord rec = new SinkRecord(SRC_TOPIC_NAME, 0, null, "key", null, value, 0L); + worker.save(ImmutableList.of(rec)); + + UUID commitId = UUID.randomUUID(); + Event commitRequest = new Event(config.connectGroupId(), new StartCommit(commitId)); + byte[] bytes = AvroUtil.encode(commitRequest); + consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 1, "key", bytes)); + + worker.process(); + + assertThat(producer.history()).hasSize(2); + + Event event = AvroUtil.decode(producer.history().get(0).value()); + assertThat(event.payload().type()).isEqualTo(PayloadType.DATA_WRITTEN); + DataWritten dataWritten = (DataWritten) event.payload(); + assertThat(dataWritten.commitId()).isEqualTo(commitId); + + event = AvroUtil.decode(producer.history().get(1).value()); + assertThat(event.type()).isEqualTo(PayloadType.DATA_COMPLETE); + DataComplete dataComplete = (DataComplete) event.payload(); + assertThat(dataComplete.commitId()).isEqualTo(commitId); + assertThat(dataComplete.assignments()).hasSize(1); + assertThat(dataComplete.assignments().get(0).offset()).isEqualTo(1L); + } + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java index 80adc7fc3e03..ac44952a5c15 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java @@ -73,7 +73,7 @@ public void before() { protected WriteResult writeTest( List<Record> rows, IcebergSinkConfig config, Class<?> expectedWriterClass) { - try (TaskWriter<Record> writer = Utilities.createTableWriter(table, "name", config)) { + try (TaskWriter<Record> writer = RecordUtils.createTableWriter(table, "name", config)) { assertThat(writer.getClass()).isEqualTo(expectedWriterClass); rows.forEach( diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java new file mode 100644 index 000000000000..08e832256a28 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.api.Test; + +public class RecordUtilsTest { + + @Test + public void testExtractFromRecordValueStruct() { + Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); + Struct val = new Struct(valSchema).put("key", 123L); + Object result = RecordUtils.extractFromRecordValue(val, "key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueStructNested() { + Schema idSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); + Schema dataSchema = SchemaBuilder.struct().field("id", idSchema).build(); + Schema valSchema = SchemaBuilder.struct().field("data", dataSchema).build(); + + Struct id = new Struct(idSchema).put("key", 123L); + Struct data = new Struct(dataSchema).put("id", id); + Struct val = new Struct(valSchema).put("data", data); + + Object result = RecordUtils.extractFromRecordValue(val, "data.id.key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueStructNull() { + Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); + Struct val = new Struct(valSchema).put("key", 123L); + + Object result = RecordUtils.extractFromRecordValue(val, ""); + assertThat(result).isNull(); + + result = RecordUtils.extractFromRecordValue(val, "xkey"); + assertThat(result).isNull(); + } + + @Test + public void testExtractFromRecordValueMap() { + Map<String, Object> val = ImmutableMap.of("key", 123L); + Object result = RecordUtils.extractFromRecordValue(val, "key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueMapNested() { + Map<String, Object> id = ImmutableMap.of("key", 123L); + Map<String, Object> data = ImmutableMap.of("id", id); + Map<String, Object> val = ImmutableMap.of("data", data); + + Object result = RecordUtils.extractFromRecordValue(val, "data.id.key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueMapNull() { + Map<String, Object> val = ImmutableMap.of("key", 123L); + + Object result = RecordUtils.extractFromRecordValue(val, ""); + assertThat(result).isNull(); + + result = RecordUtils.extractFromRecordValue(val, "xkey"); + assertThat(result).isNull(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java new file mode 100644 index 000000000000..be29ef1022a4 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class SchemaUpdateTest { + + @Test + public void testAddColumn() { + SchemaUpdate.Consumer updateConsumer = new SchemaUpdate.Consumer(); + updateConsumer.addColumn("parent", "name", Types.StringType.get()); + assertThat(updateConsumer.addColumns()).hasSize(1); + assertThat(updateConsumer.updateTypes()).isEmpty(); + assertThat(updateConsumer.makeOptionals()).isEmpty(); + + SchemaUpdate.AddColumn addColumn = updateConsumer.addColumns().iterator().next(); + assertThat(addColumn.parentName()).isEqualTo("parent"); + assertThat(addColumn.name()).isEqualTo("name"); + assertThat(addColumn.type()).isEqualTo(Types.StringType.get()); + } + + @Test + public void testUpdateType() { + SchemaUpdate.Consumer updateConsumer = new SchemaUpdate.Consumer(); + updateConsumer.updateType("name", Types.LongType.get()); + assertThat(updateConsumer.addColumns()).isEmpty(); + + assertThat(updateConsumer.updateTypes()).hasSize(1); + assertThat(updateConsumer.makeOptionals()).isEmpty(); + + SchemaUpdate.UpdateType updateType = updateConsumer.updateTypes().iterator().next(); + assertThat(updateType.name()).isEqualTo("name"); + assertThat(updateType.type()).isEqualTo(Types.LongType.get()); + } + + @Test + public void testMakeOptional() { + SchemaUpdate.Consumer updateConsumer = new SchemaUpdate.Consumer(); + updateConsumer.makeOptional("name"); + assertThat(updateConsumer.addColumns()).isEmpty(); + + assertThat(updateConsumer.updateTypes()).isEmpty(); + assertThat(updateConsumer.makeOptionals()).hasSize(1); + + SchemaUpdate.MakeOptional makeOptional = updateConsumer.makeOptionals().iterator().next(); + assertThat(makeOptional.name()).isEqualTo("name"); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java new file mode 100644 index 000000000000..4a17b926fc56 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.time.Instant; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.TableSinkConfig; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class SinkWriterTest { + + private InMemoryCatalog catalog; + + private static final Namespace NAMESPACE = Namespace.of("db"); + private static final String TABLE_NAME = "tbl"; + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(NAMESPACE, TABLE_NAME); + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + optional(3, "date", Types.StringType.get())); + private static final String ROUTE_FIELD = "fld"; + + @BeforeEach + public void before() { + catalog = initInMemoryCatalog(); + catalog.createNamespace(NAMESPACE); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + } + + @AfterEach + public void after() throws IOException { + catalog.close(); + } + + private InMemoryCatalog initInMemoryCatalog() { + InMemoryCatalog inMemoryCatalog = new InMemoryCatalog(); + inMemoryCatalog.initialize(null, ImmutableMap.of()); + return inMemoryCatalog; + } + + @Test + public void testDefaultRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + Map<String, Object> value = ImmutableMap.of(); + + List<IcebergWriterResult> writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(1); + IcebergWriterResult writerResult = writerResults.get(0); + assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + } + + @Test + public void testDefaultNoRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.tables()).thenReturn(ImmutableList.of()); + Map<String, Object> value = ImmutableMap.of(); + + List<IcebergWriterResult> writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(0); + } + + @Test + public void testStaticRoute() { + TableSinkConfig tableConfig = mock(TableSinkConfig.class); + when(tableConfig.routeRegex()).thenReturn(Pattern.compile("val")); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(tableConfig); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map<String, Object> value = ImmutableMap.of(ROUTE_FIELD, "val"); + List<IcebergWriterResult> writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(1); + IcebergWriterResult writerResult = writerResults.get(0); + assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + } + + @Test + public void testStaticNoRoute() { + TableSinkConfig tableConfig = mock(TableSinkConfig.class); + when(tableConfig.routeRegex()).thenReturn(Pattern.compile("val")); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(tableConfig); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map<String, Object> value = ImmutableMap.of(ROUTE_FIELD, "foobar"); + List<IcebergWriterResult> writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(0); + } + + @Test + public void testDynamicRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map<String, Object> value = ImmutableMap.of(ROUTE_FIELD, TABLE_IDENTIFIER.toString()); + + List<IcebergWriterResult> writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(1); + IcebergWriterResult writerResult = writerResults.get(0); + assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + } + + @Test + public void testDynamicNoRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map<String, Object> value = ImmutableMap.of(ROUTE_FIELD, "db.foobar"); + + List<IcebergWriterResult> writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(0); + } + + private List<IcebergWriterResult> sinkWriterTest( + Map<String, Object> value, IcebergSinkConfig config) { + IcebergWriterResult writeResult = + new IcebergWriterResult( + TableIdentifier.parse(TABLE_NAME), + ImmutableList.of(mock(DataFile.class)), + ImmutableList.of(), + Types.StructType.of()); + IcebergWriter writer = mock(IcebergWriter.class); + when(writer.complete()).thenReturn(ImmutableList.of(writeResult)); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + + SinkWriter sinkWriter = new SinkWriter(catalog, config); + + // save a record + Instant now = Instant.now().truncatedTo(ChronoUnit.MILLIS); + SinkRecord rec = + new SinkRecord( + "topic", + 1, + null, + "key", + null, + value, + 100L, + now.toEpochMilli(), + TimestampType.LOG_APPEND_TIME); + sinkWriter.save(ImmutableList.of(rec)); + + SinkWriterResult result = sinkWriter.completeWrite(); + + Offset offset = result.sourceOffsets().get(new TopicPartition("topic", 1)); + assertThat(offset).isNotNull(); + assertThat(offset.offset()).isEqualTo(101L); // should be 1 more than current offset + assertThat(offset.timestamp()).isEqualTo(now.atOffset(ZoneOffset.UTC)); + + return result.writerResults(); + } +} diff --git a/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java b/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java index 6558f79b93df..b6dd984a5843 100644 --- a/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java +++ b/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java @@ -34,6 +34,7 @@ public enum Support { this.lowerCaseName = name().toLowerCase(Locale.ROOT); } + @SuppressWarnings("checkstyle:ConstantName") public static final Map<String, Support> nameToSupportMap = Maps.newHashMap(); static { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index da2735774c95..8b9cd3d3d4db 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -508,6 +508,7 @@ private static void createFileForCommit(DataFile[] closedFiles, String location, LOG.debug("Iceberg committed file is created {}", fileForCommit); } + @SuppressWarnings("DangerousJavaDeserialization") private static DataFile[] readFileForCommit(String fileForCommitLocation, FileIO io) { try (ObjectInputStream ois = new ObjectInputStream(io.newInputFile(fileForCommitLocation).newStream())) { diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java index f87d79b5537e..0c698aa4b2fd 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java @@ -55,15 +55,15 @@ class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record> // <TaskAttemptId, <TABLE_NAME, HiveIcebergRecordWriter>> map to store the active writers // Stored in concurrent map, since some executor engines can share containers - private static final Map<TaskAttemptID, Map<String, HiveIcebergRecordWriter>> writers = + private static final Map<TaskAttemptID, Map<String, HiveIcebergRecordWriter>> WRITERS = Maps.newConcurrentMap(); static Map<String, HiveIcebergRecordWriter> removeWriters(TaskAttemptID taskAttemptID) { - return writers.remove(taskAttemptID); + return WRITERS.remove(taskAttemptID); } static Map<String, HiveIcebergRecordWriter> getWriters(TaskAttemptID taskAttemptID) { - return writers.get(taskAttemptID); + return WRITERS.get(taskAttemptID); } HiveIcebergRecordWriter( @@ -80,8 +80,8 @@ static Map<String, HiveIcebergRecordWriter> getWriters(TaskAttemptID taskAttempt this.io = io; this.currentKey = new PartitionKey(spec, schema); this.wrapper = new InternalRecordWrapper(schema.asStruct()); - writers.putIfAbsent(taskAttemptID, Maps.newConcurrentMap()); - writers.get(taskAttemptID).put(tableName, this); + WRITERS.putIfAbsent(taskAttemptID, Maps.newConcurrentMap()); + WRITERS.get(taskAttemptID).put(tableName, this); } @Override diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index a95454b8b0ee..a222080d71db 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.function.BiFunction; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputFormat; @@ -44,6 +45,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.StructLike; +import org.apache.iceberg.SystemConfigs; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; @@ -78,6 +80,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.SerializationUtil; +import org.apache.iceberg.util.ThreadPools; /** * Generic Mrv2 InputFormat API for Iceberg. @@ -104,7 +107,21 @@ public List<InputSplit> getSplits(JobContext context) { Optional.ofNullable( HiveIcebergStorageHandler.table(conf, conf.get(InputFormatConfig.TABLE_IDENTIFIER))) .orElseGet(() -> Catalogs.loadTable(conf)); + final ExecutorService workerPool = + ThreadPools.newWorkerPool( + "iceberg-plan-worker-pool", + conf.getInt( + SystemConfigs.WORKER_THREAD_POOL_SIZE.propertyKey(), + ThreadPools.WORKER_THREAD_POOL_SIZE)); + try { + return planInputSplits(table, conf, workerPool); + } finally { + workerPool.shutdown(); + } + } + private List<InputSplit> planInputSplits( + Table table, Configuration conf, ExecutorService workerPool) { TableScan scan = table .newScan() @@ -144,6 +161,7 @@ public List<InputSplit> getSplits(JobContext context) { InputFormatConfig.InMemoryDataModel model = conf.getEnum( InputFormatConfig.IN_MEMORY_DATA_MODEL, InputFormatConfig.InMemoryDataModel.GENERIC); + scan = scan.planWith(workerPool); try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) { Table serializableTable = SerializableTable.copyOf(table); tasksIterable.forEach( diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java index c9d5d487de38..86d390ca9ffe 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java @@ -25,11 +25,14 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.lang.reflect.Method; import java.nio.file.Path; import java.nio.file.Paths; +import java.security.PrivilegedAction; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; @@ -39,6 +42,7 @@ import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; @@ -67,6 +71,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ThreadPools; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -381,6 +386,45 @@ public void testCustomCatalog() throws IOException { testInputFormat.create(builder.conf()).validate(expectedRecords); } + @TestTemplate + public void testWorkerPool() throws Exception { + Table table = helper.createUnpartitionedTable(); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user1", new String[] {}); + UserGroupInformation user2 = + UserGroupInformation.createUserForTesting("user2", new String[] {}); + final ExecutorService workerPool1 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); + final ExecutorService workerPool2 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); + try { + assertThat(getUserFromWorkerPool(user1, table, workerPool1)).isEqualTo("user1"); + assertThat(getUserFromWorkerPool(user2, table, workerPool1)).isEqualTo("user1"); + assertThat(getUserFromWorkerPool(user2, table, workerPool2)).isEqualTo("user2"); + } finally { + workerPool1.shutdown(); + workerPool2.shutdown(); + } + } + + private String getUserFromWorkerPool( + UserGroupInformation user, Table table, ExecutorService workerpool) throws Exception { + Method method = + IcebergInputFormat.class.getDeclaredMethod( + "planInputSplits", Table.class, Configuration.class, ExecutorService.class); + method.setAccessible(true); + return user.doAs( + (PrivilegedAction<String>) + () -> { + try { + method.invoke(new IcebergInputFormat<>(), table, conf, workerpool); + return workerpool + .submit(() -> UserGroupInformation.getCurrentUser().getUserName()) + .get(); + } catch (Exception e) { + throw new RuntimeException("Failed to get user from worker pool", e); + } + }); + } + // TODO - Capture template type T in toString method: // https://github.com/apache/iceberg/issues/1542 public abstract static class TestInputFormat<T> { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java index 919230a9fbd9..3ca39c9fec36 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java @@ -41,7 +41,7 @@ public class TestHiveIcebergSerDe { - private static final Schema schema = + private static final Schema SCHEMA = new Schema(required(1, "string_field", Types.StringType.get())); @TempDir private Path tmp; @@ -58,19 +58,19 @@ public void testInitialize() throws IOException, SerDeException { properties.setProperty(InputFormatConfig.CATALOG_NAME, Catalogs.ICEBERG_HADOOP_TABLE_NAME); HadoopTables tables = new HadoopTables(conf); - tables.create(schema, location.toString()); + tables.create(SCHEMA, location.toString()); HiveIcebergSerDe serDe = new HiveIcebergSerDe(); serDe.initialize(conf, properties); - assertThat(serDe.getObjectInspector()).isEqualTo(IcebergObjectInspector.create(schema)); + assertThat(serDe.getObjectInspector()).isEqualTo(IcebergObjectInspector.create(SCHEMA)); } @Test public void testDeserialize() { HiveIcebergSerDe serDe = new HiveIcebergSerDe(); - Record record = RandomGenericData.generate(schema, 1, 0).get(0); + Record record = RandomGenericData.generate(SCHEMA, 1, 0).get(0); Container<Record> container = new Container<>(); container.set(record); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java index a7aa5126e2e2..b8a454d01f02 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java @@ -52,7 +52,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestHiveIcebergStorageHandlerTimezone { - private static final Optional<ThreadLocal<DateFormat>> dateFormat = + private static final Optional<ThreadLocal<DateFormat>> DATE_FORMAT = Optional.ofNullable( (ThreadLocal<DateFormat>) DynFields.builder() @@ -61,7 +61,7 @@ public class TestHiveIcebergStorageHandlerTimezone { .buildStatic() .get()); - private static final Optional<ThreadLocal<TimeZone>> localTimeZone = + private static final Optional<ThreadLocal<TimeZone>> LOCAL_TIME_ZONE = Optional.ofNullable( (ThreadLocal<TimeZone>) DynFields.builder() @@ -103,8 +103,8 @@ public void before() throws IOException { // Magic to clean cached date format and local timezone for Hive where the default timezone is // used/stored in the // cached object - dateFormat.ifPresent(ThreadLocal::remove); - localTimeZone.ifPresent(ThreadLocal::remove); + DATE_FORMAT.ifPresent(ThreadLocal::remove); + LOCAL_TIME_ZONE.ifPresent(ThreadLocal::remove); this.testTables = HiveIcebergStorageHandlerTestUtils.testTables( diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java index d6f4f68f511e..49b721d0e5f4 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java @@ -48,7 +48,7 @@ public class TestMultipleClients extends BaseTestIceberg { private static final String BRANCH = "multiple-clients-test"; - private static final Schema schema = + private static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); public TestMultipleClients() { @@ -153,12 +153,12 @@ public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNo @Test public void testListTables() { - createTable(TableIdentifier.parse("foo.tbl1"), schema); + createTable(TableIdentifier.parse("foo.tbl1"), SCHEMA); assertThat(catalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder(TableIdentifier.parse("foo.tbl1")); // another client creates a table with the same nessie server - anotherCatalog.createTable(TableIdentifier.parse("foo.tbl2"), schema); + anotherCatalog.createTable(TableIdentifier.parse("foo.tbl2"), SCHEMA); assertThat(anotherCatalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder( TableIdentifier.parse("foo.tbl1"), TableIdentifier.parse("foo.tbl2")); @@ -171,7 +171,7 @@ public void testListTables() { @Test public void testCommits() { TableIdentifier identifier = TableIdentifier.parse("foo.tbl1"); - createTable(identifier, schema); + createTable(identifier, SCHEMA); Table tableFromCatalog = catalog.loadTable(identifier); tableFromCatalog.updateSchema().addColumn("x1", Types.LongType.get()).commit(); @@ -188,7 +188,7 @@ public void testCommits() { @Test public void testConcurrentCommitsWithRefresh() { TableIdentifier identifier = TableIdentifier.parse("foo.tbl1"); - createTable(identifier, schema); + createTable(identifier, SCHEMA); String hashBefore = catalog.currentHash(); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index 94eb3144a0a4..f0f75c842429 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -82,9 +82,9 @@ public class TestNessieTable extends BaseTestIceberg { private static final String TABLE_NAME = "tbl"; private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME); private static final ContentKey KEY = ContentKey.of(DB_NAME, TABLE_NAME); - private static final Schema schema = + private static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - private static final Schema altered = + private static final Schema ALTERED = new Schema( Types.StructType.of( required(1, "id", Types.LongType.get()), @@ -102,7 +102,7 @@ public TestNessieTable() { public void beforeEach(NessieClientFactory clientFactory, @NessieClientUri URI nessieUri) throws IOException { super.beforeEach(clientFactory, nessieUri); - this.tableLocation = createTable(TABLE_IDENTIFIER, schema).location().replaceFirst("file:", ""); + this.tableLocation = createTable(TABLE_IDENTIFIER, SCHEMA).location().replaceFirst("file:", ""); } @Override @@ -539,7 +539,7 @@ public void testExistingTableUpdate() { // Only 2 snapshotFile Should exist and no manifests should exist assertThat(metadataVersionFiles(tableLocation)).isNotNull().hasSize(2); assertThat(manifestFiles(tableLocation)).isNotNull().isEmpty(); - assertThat(altered.asStruct()).isEqualTo(icebergTable.schema().asStruct()); + assertThat(ALTERED.asStruct()).isEqualTo(icebergTable.schema().asStruct()); } @Test @@ -614,7 +614,7 @@ public void testGCEnabledViaTableDefaultCatalogProperty() { .build()); // Create the table again using updated config defaults. - tableLocation = createTable(TABLE_IDENTIFIER, schema).location().replaceFirst("file:", ""); + tableLocation = createTable(TABLE_IDENTIFIER, SCHEMA).location().replaceFirst("file:", ""); Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); assertThatCode( @@ -675,12 +675,12 @@ protected List<String> manifestFiles(String tablePath) { private static String addRecordsToFile(Table table, String filename) throws IOException { GenericRecordBuilder recordBuilder = - new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test")); + new GenericRecordBuilder(AvroSchemaUtil.convert(SCHEMA, "test")); List<GenericData.Record> records = Lists.newArrayListWithCapacity(3); records.add(recordBuilder.set("id", 1L).build()); records.add(recordBuilder.set("id", 2L).build()); records.add(recordBuilder.set("id", 3L).build()); - return writeRecordsToFile(table, schema, filename, records); + return writeRecordsToFile(table, SCHEMA, filename, records); } } diff --git a/open-api/README.md b/open-api/README.md index 6f906ec5acef..cf13df60f7d2 100644 --- a/open-api/README.md +++ b/open-api/README.md @@ -40,3 +40,66 @@ make generate ``` The generated code is not being used in the project, but helps to see what the changes in the open-API definition are in the generated code. + +# REST Compatibility Kit (RCK) + +The REST Compatibility Kit (RCK) is a Technology Compatibility Kit (TCK) implementation for the +Iceberg REST Specification. This includes a series of tests based on the Java reference +implementation of the REST Catalog that can be executed against any REST server that implements the +spec. + +## Test Configuration + +The RCK can be configured using either environment variables or java system properties and allows +for configuring both the tests and the REST client. Environment variables prefixed by `CATALOG_` +are passed through the catalog configuring with the following mutations: + +1. The `CATALOG_` prefix is stripped from the key name +2. Single underscore (`_`) is replaced with a dot (`.`) +3. Double underscore (`__`) is replaced with a dash (`-`) +4. The key names are converted to lowercase + +A basic environment configuration would look like the following: + +```shell +CATALOG_URI=https://my_rest_server.io/ ## -> uri=https://my_rest_server.io/ +CATALOG_WAREHOUSE=test_warehouse ## -> warehouse=test_warehouse +CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO ## -> io-impl=org.apache.iceberg.aws.s3.S3FileIO +CATALOG_CREDENTIAL=<oauth_key>:<oauth_secret> ## -> credential=<oauth_key>:<oauth_secret> +``` + +Java properties passed to the test must be prefixed with `rck.`, which can be used to configure some +test configurations described below and any catalog client properties. + +An example of the same configuration using java system properties would look like the following: +```shell +rck.uri=https://my_rest_server.io/ ## -> uri=https://my_rest_server.io/ +rck.warehouse=test_warehouse ## -> warehouse=test_warehouse +rck.io-impl=org.apache.iceberg.aws.s3.S3FileIO ## -> io-impl=org.apache.iceberg.aws.s3.S3FileIO +rck.credential=<oauth_key>:<oauth_secret> ## -> credential=<oauth_key>:<oauth_secret> +``` + +Some test behaviors are configurable depending on the catalog implementations. Not all behaviors +are strictly defined by the REST Specification. The following are currently configurable: + +| config | default | +|-------------------------------|---------| +| rck.requires-namespace-create | true | +| rck.supports-serverside-retry | true | + + +## Running Compatibility Tests + +The compatibility tests can be invoked via gradle with the following: + +Note: The default behavior is to run a local http server with a jdbc backend for testing purposes, +so `-Drck.local=false` must be set to point to an external REST server. + +```shell +./gradlew :iceberg-open-api:test --tests RESTCompatibilityKitSuite \ + -Drck.local=false \ + -Drck.requires-namespace-create=true \ + -Drck.uri=https://my_rest_server.io/ \ + -Drck.warehouse=test_warehouse \ + -Drck.credential=<oauth_key>:<oauth_secret> +``` \ No newline at end of file diff --git a/open-api/requirements.txt b/open-api/requirements.txt index f6233ff2ace4..77d4f18331ca 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.25.8 +datamodel-code-generator==0.25.9 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index c5af1940c324..46d8f2aa7718 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -54,6 +54,17 @@ class CatalogConfig(BaseModel): ..., description='Properties that should be used as default configuration; applied before client configuration.', ) + endpoints: Optional[List[str]] = Field( + None, + description='A list of endpoints that the server supports. The format of each endpoint must be "<HTTP verb> <resource path from OpenAPI REST spec>". The HTTP verb and the resource path must be separated by a space character.', + example=[ + 'GET /v1/{prefix}/namespaces/{namespace}', + 'GET /v1/{prefix}/namespaces', + 'POST /v1/{prefix}/namespaces', + 'GET /v1/{prefix}/namespaces/{namespace}/tables/{table}', + 'GET /v1/{prefix}/namespaces/{namespace}/views/{view}', + ], + ) class UpdateNamespacePropertiesRequest(BaseModel): @@ -360,6 +371,11 @@ class RemovePartitionStatisticsUpdate(BaseUpdate): snapshot_id: int = Field(..., alias='snapshot-id') +class RemovePartitionSpecsUpdate(BaseUpdate): + action: Optional[Literal['remove-partition-specs']] = None + spec_ids: List[int] = Field(..., alias='spec-ids') + + class AssertCreate(BaseModel): """ The table must not already exist; used for create transactions @@ -462,6 +478,8 @@ class TokenType(BaseModel): class OAuthClientCredentialsRequest(BaseModel): """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + OAuth2 client credentials request See https://datatracker.ietf.org/doc/html/rfc6749#section-4.4 @@ -481,6 +499,8 @@ class OAuthClientCredentialsRequest(BaseModel): class OAuthTokenExchangeRequest(BaseModel): """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + OAuth2 token exchange request See https://datatracker.ietf.org/doc/html/rfc8693 @@ -500,7 +520,10 @@ class OAuthTokenExchangeRequest(BaseModel): class OAuthTokenRequest(BaseModel): - __root__: Union[OAuthClientCredentialsRequest, OAuthTokenExchangeRequest] + __root__: Union[OAuthClientCredentialsRequest, OAuthTokenExchangeRequest] = Field( + ..., + description='The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint.', + ) class CounterResult(BaseModel): @@ -532,6 +555,10 @@ class CommitReport(BaseModel): class OAuthError(BaseModel): + """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + """ + error: Literal[ 'invalid_request', 'invalid_client', @@ -545,6 +572,10 @@ class OAuthError(BaseModel): class OAuthTokenResponse(BaseModel): + """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + """ + access_token: str = Field( ..., description='The access token, for client credentials or token exchange' ) @@ -622,7 +653,7 @@ class BlobMetadata(BaseModel): snapshot_id: int = Field(..., alias='snapshot-id') sequence_number: int = Field(..., alias='sequence-number') fields: List[int] - properties: Optional[Dict[str, Any]] = None + properties: Optional[Dict[str, str]] = None class PartitionStatisticsFile(BaseModel): @@ -1009,11 +1040,9 @@ class TableMetadata(BaseModel): last_sequence_number: Optional[int] = Field(None, alias='last-sequence-number') snapshot_log: Optional[SnapshotLog] = Field(None, alias='snapshot-log') metadata_log: Optional[MetadataLog] = Field(None, alias='metadata-log') - statistics_files: Optional[List[StatisticsFile]] = Field( - None, alias='statistics-files' - ) - partition_statistics_files: Optional[List[PartitionStatisticsFile]] = Field( - None, alias='partition-statistics-files' + statistics: Optional[List[StatisticsFile]] = None + partition_statistics: Optional[List[PartitionStatisticsFile]] = Field( + None, alias='partition-statistics' ) @@ -1057,6 +1086,7 @@ class TableUpdate(BaseModel): RemovePropertiesUpdate, SetStatisticsUpdate, RemoveStatisticsUpdate, + RemovePartitionSpecsUpdate, ] diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 4bb73cd44120..eec594eb89fb 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -100,6 +100,38 @@ paths: Common catalog configuration settings are documented at https://iceberg.apache.org/docs/latest/configuration/#catalog-properties + + + The catalog configuration also holds an optional `endpoints` field that contains information about the endpoints + supported by the server. If a server does not send the `endpoints` field, a default set of endpoints is assumed: + + - GET /v1/{prefix}/namespaces + + - POST /v1/{prefix}/namespaces + + - GET /v1/{prefix}/namespaces/{namespace} + + - DELETE /v1/{prefix}/namespaces/{namespace} + + - POST /v1/{prefix}/namespaces/{namespace}/properties + + - GET /v1/{prefix}/namespaces/{namespace}/tables + + - POST /v1/{prefix}/namespaces/{namespace}/tables + + - GET /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - POST /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - DELETE /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - POST /v1/{prefix}/namespaces/{namespace}/register + + - POST /v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics + + - POST /v1/{prefix}/tables/rename + + - POST /v1/{prefix}/transactions/commit " responses: 200: @@ -114,7 +146,14 @@ paths: }, "defaults": { "clients": "4" - } + }, + "endpoints": [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] } 400: $ref: '#/components/responses/BadRequestErrorResponse' @@ -134,9 +173,22 @@ paths: post: tags: - OAuth2 API - summary: Get a token using an OAuth2 flow + summary: Get a token using an OAuth2 flow (DEPRECATED for REMOVAL) + deprecated: true operationId: getToken description: + The `oauth/tokens` endpoint is **DEPRECATED for REMOVAL**. It is _not_ recommended to + implement this endpoint, unless you are fully aware of the potential security implications. + + All clients are encouraged to explicitly set the configuration property `oauth2-server-uri` + to the correct OAuth endpoint. + + Deprecated since Iceberg (Java) 1.6.0. The endpoint and related types will be removed from + this spec in Iceberg (Java) 2.0. + + See [Security improvements in the Iceberg REST specification](https://github.com/apache/iceberg/issues/10537) + + Exchange credentials for a token using the OAuth2 client credentials flow or token exchange. @@ -673,6 +725,8 @@ paths: Commits have two parts, requirements and updates. Requirements are assertions that will be validated before attempting to make and commit changes. For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value. + Server implementations are required to fail with a 400 status code + if any unknown updates or requirements are received. Updates are changes to make to table metadata. For example, after asserting that the current main ref @@ -973,7 +1027,8 @@ paths: A commit for a single table consists of a table identifier with requirements and updates. Requirements are assertions that will be validated before attempting to make and commit changes. For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value. - + Server implementations are required to fail with a 400 status code + if any unknown updates or requirements are received. Updates are changes to make to table metadata. For example, after asserting that the current main ref is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new @@ -1567,6 +1622,19 @@ components: type: string description: Properties that should be used as default configuration; applied before client configuration. + endpoints: + type: array + items: + type: string + description: A list of endpoints that the server supports. The format of each endpoint must be "<HTTP verb> <resource path from OpenAPI REST spec>". + The HTTP verb and the resource path must be separated by a space character. + example: [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] CreateNamespaceRequest: type: object @@ -2119,11 +2187,11 @@ components: metadata-log: $ref: '#/components/schemas/MetadataLog' # statistics - statistics-files: + statistics: type: array items: $ref: '#/components/schemas/StatisticsFile' - partition-statistics-files: + partition-statistics: type: array items: $ref: '#/components/schemas/PartitionStatisticsFile' @@ -2252,6 +2320,7 @@ components: remove-statistics: '#/components/schemas/RemoveStatisticsUpdate' set-partition-statistics: '#/components/schemas/SetPartitionStatisticsUpdate' remove-partition-statistics: '#/components/schemas/RemovePartitionStatisticsUpdate' + remove-partition-specs: '#/components/schemas/RemovePartitionSpecsUpdate' type: object required: - action @@ -2554,6 +2623,20 @@ components: type: integer format: int64 + RemovePartitionSpecsUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + required: + - spec-ids + properties: + action: + type: string + enum: [ "remove-partition-specs" ] + spec-ids: + type: array + items: + type: integer + TableUpdate: anyOf: - $ref: '#/components/schemas/AssignUUIDUpdate' @@ -2573,6 +2656,7 @@ components: - $ref: '#/components/schemas/RemovePropertiesUpdate' - $ref: '#/components/schemas/SetStatisticsUpdate' - $ref: '#/components/schemas/RemoveStatisticsUpdate' + - $ref: '#/components/schemas/RemovePartitionSpecsUpdate' ViewUpdate: anyOf: @@ -2922,7 +3006,12 @@ components: See https://datatracker.ietf.org/doc/html/rfc8693#section-3 OAuthClientCredentialsRequest: - description: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. + + OAuth2 client credentials request @@ -2957,7 +3046,12 @@ components: a Basic Authorization header. OAuthTokenExchangeRequest: + deprecated: true description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. + + OAuth2 token exchange request @@ -2988,6 +3082,10 @@ components: $ref: '#/components/schemas/TokenType' OAuthTokenRequest: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. anyOf: - $ref: '#/components/schemas/OAuthClientCredentialsRequest' - $ref: '#/components/schemas/OAuthTokenExchangeRequest' @@ -3142,6 +3240,10 @@ components: type: string OAuthError: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. type: object required: - error @@ -3161,6 +3263,10 @@ components: type: string OAuthTokenResponse: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. type: object required: - access_token @@ -3354,6 +3460,8 @@ components: type: integer properties: type: object + additionalProperties: + type: string PartitionStatisticsFile: type: object diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java new file mode 100644 index 000000000000..4c4860e88a19 --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.catalog.CatalogTests; +import org.apache.iceberg.util.PropertyUtil; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(RESTServerExtension.class) +public class RESTCompatibilityKitCatalogTests extends CatalogTests<RESTCatalog> { + private static final Logger LOG = LoggerFactory.getLogger(RESTCompatibilityKitCatalogTests.class); + + private static RESTCatalog restCatalog; + + @BeforeAll + static void beforeClass() throws Exception { + restCatalog = RCKUtils.initCatalogClient(); + + assertThat(restCatalog.listNamespaces()) + .withFailMessage("Namespaces list should not contain: %s", RCKUtils.TEST_NAMESPACES) + .doesNotContainAnyElementsOf(RCKUtils.TEST_NAMESPACES); + } + + @BeforeEach + void before() { + try { + RCKUtils.purgeCatalogTestEntries(restCatalog); + } catch (Exception e) { + LOG.warn("Failure during test setup", e); + } + } + + @AfterAll + static void afterClass() throws Exception { + restCatalog.close(); + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_REQUIRES_NAMESPACE_CREATE, + super.requiresNamespaceCreate()); + } + + @Override + protected boolean supportsServerSideRetry() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_SUPPORTS_SERVERSIDE_RETRY, true); + } + + @Override + protected boolean overridesRequestedLocation() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_OVERRIDES_REQUESTED_LOCATION, + false); + } +} diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java new file mode 100644 index 000000000000..a7bbe64c0650 --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import org.junit.platform.suite.api.SelectClasses; +import org.junit.platform.suite.api.Suite; +import org.junit.platform.suite.api.SuiteDisplayName; + +/** + * Iceberg REST Compatibility Kit + * + * <p>This test suite provides the ability to run the Iceberg catalog tests against a remote REST + * catalog implementation to verify the behaviors against the reference implementation catalog + * tests. + * + * <p>The tests can be configured through environment variables or system properties. By default, + * the tests will run using a local http server using a servlet implementation that leverages the + * {@link RESTCatalogAdapter}. + */ +@Suite +@SuiteDisplayName("Iceberg REST Compatibility Kit") +@SelectClasses({RESTCompatibilityKitCatalogTests.class, RESTCompatibilityKitViewCatalogTests.class}) +public class RESTCompatibilityKitSuite { + static final String RCK_REQUIRES_NAMESPACE_CREATE = "rck.requires-namespace-create"; + static final String RCK_SUPPORTS_SERVERSIDE_RETRY = "rck.supports-serverside-retry"; + static final String RCK_OVERRIDES_REQUESTED_LOCATION = "rck.overrides-requested-location"; + + protected RESTCompatibilityKitSuite() {} +} diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java new file mode 100644 index 000000000000..7a18531bf6af --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.ViewCatalogTests; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(RESTServerExtension.class) +public class RESTCompatibilityKitViewCatalogTests extends ViewCatalogTests<RESTCatalog> { + private static final Logger LOG = + LoggerFactory.getLogger(RESTCompatibilityKitViewCatalogTests.class); + private static RESTCatalog restCatalog; + + @BeforeAll + static void beforeClass() throws Exception { + restCatalog = RCKUtils.initCatalogClient(); + + assertThat(restCatalog.listNamespaces()) + .withFailMessage("Namespaces list should not contain: %s", RCKUtils.TEST_NAMESPACES) + .doesNotContainAnyElementsOf(RCKUtils.TEST_NAMESPACES); + } + + @BeforeEach + void before() { + try { + RCKUtils.purgeCatalogTestEntries(restCatalog); + } catch (Exception e) { + LOG.warn("Failure during test setup", e); + } + } + + @AfterAll + static void afterClass() throws Exception { + restCatalog.close(); + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected Catalog tableCatalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_REQUIRES_NAMESPACE_CREATE, true); + } + + @Override + protected boolean supportsServerSideRetry() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_SUPPORTS_SERVERSIDE_RETRY, true); + } + + @Override + protected boolean overridesRequestedLocation() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_OVERRIDES_REQUESTED_LOCATION, + false); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java new file mode 100644 index 000000000000..0f1571d362e3 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; + +class RCKUtils { + private static final String CATALOG_ENV_PREFIX = "CATALOG_"; + static final String RCK_LOCAL = "rck.local"; + static final String RCK_PURGE_TEST_NAMESPACES = "rck.purge-test-namespaces"; + + static final List<Namespace> TEST_NAMESPACES = List.of(Namespace.of("ns"), Namespace.of("newdb")); + + private RCKUtils() {} + + /** + * Utility method that allows configuring catalog properties via environment variables. + * + * <p>Returns a property map for all environment variables that start with <code>CATALOG_</code> + * replacing double-underscore (<code>__</code>) with dash (<code>-</code>) and replacing single + * underscore (<code>_</code>) with dot (<code>.</code>) to allow for common catalog property + * conventions. All characters in the name are converted to lowercase and values are unmodified. + * + * <p>Examples: + * + * <pre><code> + * CATALOG_CATALOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog -> catalog-impl=org.apache.iceberg.jdbc.JdbcCatalog + * CATALOG_URI=jdbc:sqlite:memory: -> uri=jdbc:sqlite:memory: + * CATALOG_WAREHOUSE=test_warehouse -> warehouse=test_warehouse + * CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO -> io-impl=org.apache.iceberg.aws.s3.S3FileIO + * CATALOG_JDBC_USER=ice_user -> jdbc.user=ice_user + * </code></pre> + * + * @return configuration map + */ + static Map<String, String> environmentCatalogConfig() { + return System.getenv().entrySet().stream() + .filter(e -> e.getKey().startsWith(CATALOG_ENV_PREFIX)) + .collect( + Collectors.toMap( + e -> + e.getKey() + .replaceFirst(CATALOG_ENV_PREFIX, "") + .replaceAll("__", "-") + .replaceAll("_", ".") + .toLowerCase(Locale.ROOT), + Map.Entry::getValue, + (m1, m2) -> { + throw new IllegalArgumentException("Duplicate key: " + m1); + }, + HashMap::new)); + } + + static RESTCatalog initCatalogClient() { + Map<String, String> catalogProperties = Maps.newHashMap(); + catalogProperties.putAll(RCKUtils.environmentCatalogConfig()); + catalogProperties.putAll(Maps.fromProperties(System.getProperties())); + + // Set defaults + catalogProperties.putIfAbsent( + CatalogProperties.URI, + String.format("http://localhost:%s/", RESTCatalogServer.REST_PORT_DEFAULT)); + catalogProperties.putIfAbsent(CatalogProperties.WAREHOUSE_LOCATION, "rck_warehouse"); + + RESTCatalog catalog = new RESTCatalog(); + catalog.setConf(new Configuration()); + catalog.initialize("rck_catalog", catalogProperties); + return catalog; + } + + static void purgeCatalogTestEntries(RESTCatalog catalog) { + if (!PropertyUtil.propertyAsBoolean(catalog.properties(), RCK_PURGE_TEST_NAMESPACES, true)) { + return; + } + + TEST_NAMESPACES.stream() + .filter(catalog::namespaceExists) + .forEach( + namespace -> { + catalog.listTables(namespace).forEach(catalog::dropTable); + catalog.listViews(namespace).forEach(catalog::dropView); + catalog.dropNamespace(namespace); + }); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java new file mode 100644 index 000000000000..b3d12f74e4b0 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.util.PropertyUtil; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RESTCatalogServer { + private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServer.class); + + static final String REST_PORT = "rest.port"; + static final int REST_PORT_DEFAULT = 8181; + + private Server httpServer; + + RESTCatalogServer() {} + + static class CatalogContext { + private final Catalog catalog; + private final Map<String, String> configuration; + + CatalogContext(Catalog catalog, Map<String, String> configuration) { + this.catalog = catalog; + this.configuration = configuration; + } + + public Catalog catalog() { + return catalog; + } + + public Map<String, String> configuration() { + return configuration; + } + } + + private CatalogContext initializeBackendCatalog() throws IOException { + // Translate environment variables to catalog properties + Map<String, String> catalogProperties = RCKUtils.environmentCatalogConfig(); + + // Fallback to a JDBCCatalog impl if one is not set + catalogProperties.putIfAbsent(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); + catalogProperties.putIfAbsent(CatalogProperties.URI, "jdbc:sqlite::memory:"); + catalogProperties.putIfAbsent("jdbc.schema-version", "V1"); + + // Configure a default location if one is not specified + String warehouseLocation = catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION); + + if (warehouseLocation == null) { + File tmp = java.nio.file.Files.createTempDirectory("iceberg_warehouse").toFile(); + tmp.deleteOnExit(); + warehouseLocation = tmp.toPath().resolve("iceberg_data").toFile().getAbsolutePath(); + catalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + LOG.info("No warehouse location set. Defaulting to temp location: {}", warehouseLocation); + } + + LOG.info("Creating catalog with properties: {}", catalogProperties); + return new CatalogContext( + CatalogUtil.buildIcebergCatalog("rest_backend", catalogProperties, new Configuration()), + catalogProperties); + } + + public void start(boolean join) throws Exception { + CatalogContext catalogContext = initializeBackendCatalog(); + + RESTCatalogAdapter adapter = new RESTServerCatalogAdapter(catalogContext); + RESTCatalogServlet servlet = new RESTCatalogServlet(adapter); + + ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + ServletHolder servletHolder = new ServletHolder(servlet); + context.addServlet(servletHolder, "/*"); + context.insertHandler(new GzipHandler()); + + this.httpServer = + new Server( + PropertyUtil.propertyAsInt(catalogContext.configuration, REST_PORT, REST_PORT_DEFAULT)); + httpServer.setHandler(context); + httpServer.start(); + + if (join) { + httpServer.join(); + } + } + + public void stop() throws Exception { + if (httpServer != null) { + httpServer.stop(); + } + } + + public static void main(String[] args) throws Exception { + new RESTCatalogServer().start(true); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java new file mode 100644 index 000000000000..612ed926f54d --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.util.Map; +import org.apache.iceberg.aws.s3.S3FileIOProperties; +import org.apache.iceberg.azure.AzureProperties; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.rest.RESTCatalogServer.CatalogContext; +import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.util.PropertyUtil; + +class RESTServerCatalogAdapter extends RESTCatalogAdapter { + private static final String INCLUDE_CREDENTIALS = "include-credentials"; + + private final CatalogContext catalogContext; + + RESTServerCatalogAdapter(CatalogContext catalogContext) { + super(catalogContext.catalog()); + this.catalogContext = catalogContext; + } + + @Override + public <T extends RESTResponse> T handleRequest( + Route route, Map<String, String> vars, Object body, Class<T> responseType) { + T restResponse = super.handleRequest(route, vars, body, responseType); + + if (restResponse instanceof LoadTableResponse) { + if (PropertyUtil.propertyAsBoolean( + catalogContext.configuration(), INCLUDE_CREDENTIALS, false)) { + applyCredentials( + catalogContext.configuration(), ((LoadTableResponse) restResponse).config()); + } + } + + return restResponse; + } + + private void applyCredentials( + Map<String, String> catalogConfig, Map<String, String> tableConfig) { + if (catalogConfig.containsKey(S3FileIOProperties.ACCESS_KEY_ID)) { + tableConfig.put( + S3FileIOProperties.ACCESS_KEY_ID, catalogConfig.get(S3FileIOProperties.ACCESS_KEY_ID)); + } + + if (catalogConfig.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { + tableConfig.put( + S3FileIOProperties.SECRET_ACCESS_KEY, + catalogConfig.get(S3FileIOProperties.SECRET_ACCESS_KEY)); + } + + if (catalogConfig.containsKey(S3FileIOProperties.SESSION_TOKEN)) { + tableConfig.put( + S3FileIOProperties.SESSION_TOKEN, catalogConfig.get(S3FileIOProperties.SESSION_TOKEN)); + } + + if (catalogConfig.containsKey(GCPProperties.GCS_OAUTH2_TOKEN)) { + tableConfig.put( + GCPProperties.GCS_OAUTH2_TOKEN, catalogConfig.get(GCPProperties.GCS_OAUTH2_TOKEN)); + } + + catalogConfig.entrySet().stream() + .filter( + entry -> + entry.getKey().startsWith(AzureProperties.ADLS_SAS_TOKEN_PREFIX) + || entry.getKey().startsWith(AzureProperties.ADLS_CONNECTION_STRING_PREFIX)) + .forEach(entry -> tableConfig.put(entry.getKey(), entry.getValue())); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java new file mode 100644 index 000000000000..fc3648055694 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class RESTServerExtension implements BeforeAllCallback, AfterAllCallback { + private RESTCatalogServer localServer; + + @Override + public void beforeAll(ExtensionContext extensionContext) throws Exception { + if (Boolean.parseBoolean( + extensionContext.getConfigurationParameter(RCKUtils.RCK_LOCAL).orElse("true"))) { + this.localServer = new RESTCatalogServer(); + this.localServer.start(false); + } + } + + @Override + public void afterAll(ExtensionContext extensionContext) throws Exception { + if (localServer != null) { + localServer.stop(); + } + } +} diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java index 2da9b2b9deba..62f330f9f572 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java @@ -82,7 +82,7 @@ public class TestBloomRowGroupFilter { - private static final Types.StructType structFieldType = + private static final Types.StructType STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get())); private static final Schema SCHEMA = new Schema( @@ -100,7 +100,7 @@ public class TestBloomRowGroupFilter { optional(12, "all_nans", DoubleType.get()), optional(13, "some_nans", FloatType.get()), optional(14, "no_nans", DoubleType.get()), - optional(15, "struct_not_null", structFieldType), + optional(15, "struct_not_null", STRUCT_FIELD_TYPE), optional(17, "not_in_file", FloatType.get()), optional(18, "no_stats", StringType.get()), optional(19, "boolean", Types.BooleanType.get()), @@ -113,7 +113,7 @@ public class TestBloomRowGroupFilter { optional(26, "long_decimal", Types.DecimalType.of(14, 2)), optional(27, "fixed_decimal", Types.DecimalType.of(31, 2))); - private static final Types.StructType _structFieldType = + private static final Types.StructType UNDERSCORE_STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get())); private static final Schema FILE_SCHEMA = @@ -132,7 +132,7 @@ public class TestBloomRowGroupFilter { optional(12, "_all_nans", DoubleType.get()), optional(13, "_some_nans", FloatType.get()), optional(14, "_no_nans", DoubleType.get()), - optional(15, "_struct_not_null", _structFieldType), + optional(15, "_struct_not_null", UNDERSCORE_STRUCT_FIELD_TYPE), optional(18, "_no_stats", StringType.get()), optional(19, "_boolean", Types.BooleanType.get()), optional(20, "_time", Types.TimeType.get()), @@ -161,7 +161,7 @@ public class TestBloomRowGroupFilter { private static final double DOUBLE_BASE = 1000D; private static final float FLOAT_BASE = 10000F; private static final String BINARY_PREFIX = "BINARY测试_"; - private static final Instant instant = Instant.parse("2018-10-10T00:00:00.000Z"); + private static final Instant INSTANT = Instant.parse("2018-10-10T00:00:00.000Z"); private static final List<UUID> RANDOM_UUIDS; private static final List<byte[]> RANDOM_BYTES; @@ -192,7 +192,7 @@ public void createInputFile() throws IOException { assertThat(temp.delete()).isTrue(); // build struct field schema - org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); + org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); OutputFile outFile = Files.localOutput(temp); try (FileAppender<Record> appender = @@ -251,10 +251,10 @@ public void createInputFile() throws IOException { builder.set("_struct_not_null", structNotNull); // struct with int builder.set("_no_stats", TOO_LONG_FOR_STATS); // value longer than 4k will produce no stats builder.set("_boolean", i % 2 == 0); - builder.set("_time", instant.plusSeconds(i * 86400).toEpochMilli()); - builder.set("_date", instant.plusSeconds(i * 86400).getEpochSecond()); - builder.set("_timestamp", instant.plusSeconds(i * 86400).toEpochMilli()); - builder.set("_timestamptz", instant.plusSeconds(i * 86400).toEpochMilli()); + builder.set("_time", INSTANT.plusSeconds(i * 86400).toEpochMilli()); + builder.set("_date", INSTANT.plusSeconds(i * 86400).getEpochSecond()); + builder.set("_timestamp", INSTANT.plusSeconds(i * 86400).toEpochMilli()); + builder.set("_timestamptz", INSTANT.plusSeconds(i * 86400).toEpochMilli()); builder.set("_binary", RANDOM_BYTES.get(i)); builder.set("_int_decimal", new BigDecimal(String.valueOf(77.77 + i))); builder.set("_long_decimal", new BigDecimal(String.valueOf(88.88 + i))); @@ -807,7 +807,7 @@ public void testBooleanEq() { @Test public void testTimeEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("time", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); @@ -822,7 +822,7 @@ public void testTimeEq() { @Test public void testDateEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("date", ins.getEpochSecond())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); @@ -837,7 +837,7 @@ public void testDateEq() { @Test public void testTimestampEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("timestamp", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); @@ -852,7 +852,7 @@ public void testTimestampEq() { @Test public void testTimestamptzEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("timestamptz", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java index d690d3cf5175..fe3e8a402bb5 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java @@ -90,7 +90,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDictionaryRowGroupFilter { - private static final Types.StructType structFieldType = + private static final Types.StructType STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(9, "int_field", IntegerType.get())); private static final Schema SCHEMA = @@ -102,7 +102,7 @@ public class TestDictionaryRowGroupFilter { optional(5, "some_nulls", StringType.get()), optional(6, "no_nulls", StringType.get()), optional(7, "non_dict", StringType.get()), - optional(8, "struct_not_null", structFieldType), + optional(8, "struct_not_null", STRUCT_FIELD_TYPE), optional(10, "not_in_file", FloatType.get()), optional(11, "all_nans", DoubleType.get()), optional(12, "some_nans", FloatType.get()), @@ -113,7 +113,7 @@ public class TestDictionaryRowGroupFilter { DecimalType.of(20, 10)), // >18 precision to enforce FIXED_LEN_BYTE_ARRAY optional(15, "_nans_and_nulls", DoubleType.get())); - private static final Types.StructType _structFieldType = + private static final Types.StructType UNDERSCORE_STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(9, "_int_field", IntegerType.get())); private static final Schema FILE_SCHEMA = @@ -125,7 +125,7 @@ public class TestDictionaryRowGroupFilter { optional(5, "_some_nulls", StringType.get()), optional(6, "_no_nulls", StringType.get()), optional(7, "_non_dict", StringType.get()), - optional(8, "_struct_not_null", _structFieldType), + optional(8, "_struct_not_null", UNDERSCORE_STRUCT_FIELD_TYPE), optional(11, "_all_nans", DoubleType.get()), optional(12, "_some_nans", FloatType.get()), optional(13, "_no_nans", DoubleType.get()), @@ -171,7 +171,7 @@ public void createInputFile() throws IOException { assertThat(parquetFile.delete()).isTrue(); // build struct field schema - org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); + org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); OutputFile outFile = Files.localOutput(parquetFile); try (FileAppender<Record> appender = diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java index ea47ecb1c268..63f512ee632b 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java @@ -46,36 +46,36 @@ public class TestParquetEncryption { - private static final String columnName = "intCol"; - private static final int recordCount = 100; - private static final ByteBuffer fileDek = ByteBuffer.allocate(16); - private static final ByteBuffer aadPrefix = ByteBuffer.allocate(16); + private static final String COLUMN_NAME = "intCol"; + private static final int RECORD_COUNT = 100; + private static final ByteBuffer FILE_DEK = ByteBuffer.allocate(16); + private static final ByteBuffer AAD_PREFIX = ByteBuffer.allocate(16); + private static final Schema SCHEMA = new Schema(optional(1, COLUMN_NAME, IntegerType.get())); private static File file; - private static final Schema schema = new Schema(optional(1, columnName, IntegerType.get())); @TempDir private Path temp; @BeforeEach public void writeEncryptedFile() throws IOException { - List<GenericData.Record> records = Lists.newArrayListWithCapacity(recordCount); - org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - for (int i = 1; i <= recordCount; i++) { + List<GenericData.Record> records = Lists.newArrayListWithCapacity(RECORD_COUNT); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(SCHEMA.asStruct()); + for (int i = 1; i <= RECORD_COUNT; i++) { GenericData.Record record = new GenericData.Record(avroSchema); - record.put(columnName, i); + record.put(COLUMN_NAME, i); records.add(record); } SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); file = createTempFile(temp); FileAppender<GenericData.Record> writer = Parquet.write(localOutput(file)) - .schema(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .schema(SCHEMA) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .build(); try (Closeable toClose = writer) { @@ -86,7 +86,7 @@ public void writeEncryptedFile() throws IOException { @Test public void testReadEncryptedFileWithoutKeys() throws IOException { assertThatThrownBy( - () -> Parquet.read(localInput(file)).project(schema).callInit().build().iterator()) + () -> Parquet.read(localInput(file)).project(SCHEMA).callInit().build().iterator()) .as("Decrypted without keys") .isInstanceOf(ParquetCryptoRuntimeException.class) .hasMessage("Trying to read file with encrypted footer. No keys available"); @@ -97,8 +97,8 @@ public void testReadEncryptedFileWithoutAADPrefix() throws IOException { assertThatThrownBy( () -> Parquet.read(localInput(file)) - .project(schema) - .withFileEncryptionKey(fileDek) + .project(SCHEMA) + .withFileEncryptionKey(FILE_DEK) .callInit() .build() .iterator()) @@ -113,15 +113,15 @@ public void testReadEncryptedFileWithoutAADPrefix() throws IOException { public void testReadEncryptedFile() throws IOException { try (CloseableIterator readRecords = Parquet.read(localInput(file)) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) - .project(schema) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) + .project(SCHEMA) .callInit() .build() .iterator()) { - for (int i = 1; i <= recordCount; i++) { + for (int i = 1; i <= RECORD_COUNT; i++) { GenericData.Record readRecord = (GenericData.Record) readRecords.next(); - assertThat(readRecord.get(columnName)).isEqualTo(i); + assertThat(readRecord.get(COLUMN_NAME)).isEqualTo(i); } } } diff --git a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java index 5f3d3ac665fd..88233c58a372 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java @@ -70,8 +70,8 @@ public class IcebergStorage extends LoadFunc public static final String PIG_ICEBERG_TABLES_IMPL = "pig.iceberg.tables.impl"; private static Tables iceberg; - private static final Map<String, Table> tables = Maps.newConcurrentMap(); - private static final Map<String, String> locations = Maps.newConcurrentMap(); + private static final Map<String, Table> TABLES = Maps.newConcurrentMap(); + private static final Map<String, String> LOCATIONS = Maps.newConcurrentMap(); private String signature; @@ -81,7 +81,7 @@ public class IcebergStorage extends LoadFunc public void setLocation(String location, Job job) { LOG.info("[{}]: setLocation() -> {}", signature, location); - locations.put(signature, location); + LOCATIONS.put(signature, location); Configuration conf = job.getConfiguration(); @@ -93,9 +93,9 @@ public void setLocation(String location, Job job) { @Override public InputFormat getInputFormat() { LOG.info("[{}]: getInputFormat()", signature); - String location = locations.get(signature); + String location = LOCATIONS.get(signature); - return new IcebergPigInputFormat(tables.get(location), signature); + return new IcebergPigInputFormat(TABLES.get(location), signature); } @Override @@ -323,13 +323,13 @@ private Table load(String location, Job job) throws IOException { iceberg = (Tables) ReflectionUtils.newInstance(tablesImpl, job.getConfiguration()); } - Table result = tables.get(location); + Table result = TABLES.get(location); if (result == null) { try { LOG.info("[{}]: Loading table for location: {}", signature, location); result = iceberg.load(location); - tables.put(location, result); + TABLES.put(location, result); } catch (Exception e) { throw new FrontendException("Failed to instantiate tables implementation", e); } diff --git a/settings.gradle b/settings.gradle index 46c85fb65546..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}" @@ -198,3 +198,7 @@ project(":iceberg-kafka-connect:kafka-connect-events").name = "iceberg-kafka-con include ":iceberg-kafka-connect:kafka-connect" project(":iceberg-kafka-connect:kafka-connect").projectDir = file('kafka-connect/kafka-connect') project(":iceberg-kafka-connect:kafka-connect").name = "iceberg-kafka-connect" + +include ":iceberg-kafka-connect:kafka-connect-runtime" +project(":iceberg-kafka-connect:kafka-connect-runtime").projectDir = file('kafka-connect/kafka-connect-runtime') +project(":iceberg-kafka-connect:kafka-connect-runtime").name = "iceberg-kafka-connect-runtime" diff --git a/site/README.md b/site/README.md index 39bc1d931b81..97ddddc5cd69 100644 --- a/site/README.md +++ b/site/README.md @@ -113,6 +113,16 @@ To clear all build files, run `clean`. make clean ``` +#### Testing local changes on versioned docs + +When you build the docs as described above, by default the versioned docs are mounted from the upstream remote repositiory called `iceberg_docs`. One exception is the `nightly` version that is a soft link to the local `docs/` folder. + +When you make changes to some of the historical versioned docs in a local git branch you can mount this git branch instead of the remote one by setting the following environment variables: + +`ICEBERG_VERSIONED_DOCS_BRANCH` for the `docs/` folder + +`ICEBERG_VERSIONED_JAVADOC_BRANCH` for the `javadoc/` folder + #### Offline mode One of the great advantages to the MkDocs material plugin is the [offline feature](https://squidfunk.github.io/mkdocs-material/plugins/offline). You can view the Iceberg docs without the need of a server. To enable OFFLINE builds, add theOFFLINE environment variable to either `build` or `serve` recipes. diff --git a/site/dev/common.sh b/site/dev/common.sh index 481628aa4dbd..6fc045560f39 100755 --- a/site/dev/common.sh +++ b/site/dev/common.sh @@ -197,9 +197,12 @@ pull_versioned_docs () { # Ensure the remote repository for documentation exists and is up-to-date create_or_update_docs_remote - # Add local worktrees for documentation and javadoc from the remote repository - git worktree add -f docs/docs "${REMOTE}/docs" - git worktree add -f docs/javadoc "${REMOTE}/javadoc" + # Add local worktrees for documentation and javadoc either from the remote repository + # or from a local branch. + local docs_branch="${ICEBERG_VERSIONED_DOCS_BRANCH:-${REMOTE}/docs}" + local javadoc_branch="${ICEBERG_VERSIONED_JAVADOC_BRANCH:-${REMOTE}/javadoc}" + git worktree add -f docs/docs "${docs_branch}" + git worktree add -f docs/javadoc "${javadoc_branch}" # Retrieve the latest version of documentation for processing local latest_version=$(get_latest_version) diff --git a/site/docs/benchmarks.md b/site/docs/benchmarks.md index 636ad36b1646..a74b65d48f4e 100644 --- a/site/docs/benchmarks.md +++ b/site/docs/benchmarks.md @@ -27,7 +27,7 @@ Also note that JMH benchmarks run within the same JVM as the system-under-test, It is possible to run one or more Benchmarks via the **JMH Benchmarks** GH action on your own fork of the Iceberg repo. This GH action takes the following inputs: * The repository name where those benchmarks should be run against, such as `apache/iceberg` or `<user>/iceberg` -* The branch name to run benchmarks against, such as `master` or `my-cool-feature-branch` +* The branch name to run benchmarks against, such as `main` or `my-cool-feature-branch` * A list of comma-separated double-quoted Benchmark names, such as `"IcebergSourceFlatParquetDataReadBenchmark", "IcebergSourceFlatParquetDataFilterBenchmark", "IcebergSourceNestedListParquetDataWriteBenchmark"` Benchmark results will be uploaded once **all** benchmarks are done. diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 73eac3cf710a..00b0688e86a0 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -215,6 +215,12 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ron Ortloff](https://www.linkedin.com/in/ron-ortloff/), [Dennis Huo](https://www.linkedin.com/in/dennis-huo-2aaba92a/) +<!-- markdown-link-check-disable-next-line --> +### [Open Data Lakehouse powered by Apache Iceberg on Apache Ozone](https://medium.com/engineering-cloudera/open-data-lakehouse-powered-by-apache-iceberg-on-apache-ozone-a225d5dcfe98/) +**Date**: February 28th, 2023, **Company**: Cloudera + +**Authors**: [Saketa Chalamchala](https://www.linkedin.com/in/saketa-chalamchala-3602026a) + <!-- markdown-link-check-disable-next-line --> ### [Dealing with Data Incidents Using the Rollback Feature in Apache Iceberg](https://www.dremio.com/blog/dealing-with-data-incidents-using-the-rollback-feature-in-apache-iceberg/) **Date**: February 24th, 2022, **Company**: Dremio diff --git a/site/docs/concepts/catalog.md b/site/docs/concepts/catalog.md index fdd664f0cf05..ee0de3b93836 100644 --- a/site/docs/concepts/catalog.md +++ b/site/docs/concepts/catalog.md @@ -26,7 +26,7 @@ You may think of Iceberg as a format for managing data in a single table, but th The first step when using an Iceberg client is almost always initializing and configuring a catalog. The configured catalog is then used by compute engines to execute catalog operations. Multiple types of compute engines using a shared Iceberg catalog allows them to share a common data layer. -A catalog is almost always configured through the processing engine which passes along a set of properties during initialization. Different processing engines have different ways to configure a catalog. When configuring a catalog, it’s always best to refer to the [Iceberg documentation](docs/latest/configuration.md#catalog-properties) as well as the docs for the specific processing engine being used. Ultimately, these configurations boil down to a common set of catalog properties that will be passed to configure the Iceberg catalog. +A catalog is almost always configured through the processing engine which passes along a set of properties during initialization. Different processing engines have different ways to configure a catalog. When configuring a catalog, it’s always best to refer to the [Iceberg documentation](../docs/latest/configuration.md#catalog-properties) as well as the docs for the specific processing engine being used. Ultimately, these configurations boil down to a common set of catalog properties that will be passed to configure the Iceberg catalog. ## Catalog Implementations @@ -43,7 +43,7 @@ There are more catalog types in addition to the ones listed here as well as cust ## Decoupling Using the REST Catalog -The REST catalog was introduced in the Iceberg 0.14.0 release and provides greater control over how Iceberg catalogs are implemented. Instead of using technology-specific logic contained in the catalog clients, the implementation details of a REST catalog lives on the catalog server. If you’re familiar with Hive, this is somewhat similar to the Hive thrift service that allows access to a hive server over a single port. The server-side logic can be written in any language and use any custom technology, as long as the API follows the [Iceberg REST Open API specification](https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml). +The REST catalog was introduced in the Iceberg 0.14.0 release and provides greater control over how Iceberg catalogs are implemented. Instead of using technology-specific logic contained in the catalog clients, the implementation details of a REST catalog lives on the catalog server. If you’re familiar with Hive, this is somewhat similar to the Hive thrift service that allows access to a hive server over a single port. The server-side logic can be written in any language and use any custom technology, as long as the API follows the [Iceberg REST Open API specification](https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml). A great benefit of the REST catalog is that it allows you to use a single client to talk to any catalog backend. This increased flexibility makes it easier to make custom catalogs compatible with engines like Athena or Starburst without requiring the inclusion of a Jar into the classpath. diff --git a/site/docs/contribute.md b/site/docs/contribute.md index d1e16e1c9939..beda3cfc507e 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -45,6 +45,18 @@ The Iceberg community prefers to receive contributions as [Github pull requests] * If a PR is related to an issue, adding `Closes #1234` in the PR description will automatically close the issue and helps keep the project clean * If a PR is posted for visibility and isn't necessarily ready for review or merging, be sure to convert the PR to a draft +### Merging Pull Requests + +Most pull requests can be merged once a single [committer](https://www.apache.org/foundation/how-it-works/#committers) other than the author is satisfied with the code in the PR (exceptions that require additional input from the community are detailed below). [Committers are trusted](https://infra.apache.org/new-committers-guide.html#the-committers-way) to act in the best [interest of the project](https://community.apache.org/projectIndependence.html#apache-projects-are-managed-independently). + +Before merging all review comments should be addressed either by making changes or agreeing the request is out of scope for the PR. For additions to public APIs committers should wait at least 24 hours before merging to ensure there is no additional feedback from members of the community. + +Requesting changes on a PR indicates a reviewer believes the PR has merit but still needs issues addressed before merging. If a reviewer believes the change should not be merged at all and there is nothing the author could do to address the reviewers concerns, the reviewer should explicitly state this on the PR. In the rare event that a PR author and reviewers cannot come to a consensus on a PR, the disagreement should be raised to the developer mailing list for further discussion. In this context, a reviewer is anyone leaving comments on the PR including contributors, committers and PMC members. + +There are several exceptions to a single committer being able to merge a PR: + +* Behavioral and functional changes to a specification must go through the [Iceberg improvement proposal](#apache-iceberg-improvement-proposals) before any code can be merged. +* Changes to files under the `format` directory and `open-api/rest-catalog*` are considered specification changes. Unless already covered under an Iceberg improvement proposal, specification changes require their own vote (e.g. bug fixes or specification clarifications). The vote follows the ASF [code modification](https://www.apache.org/foundation/voting.html#votes-on-code-modification) model and no lazy consensus modifier. Grammar, spelling and minor formatting fixes are exempted from this rule. Draft specifications (new independent specifications that are going through the Iceberg improvement process) do not require a vote but authors should provide notice on the developer mailing list about substantive changes (the final draft will be subject to a vote). ## Apache Iceberg Improvement Proposals @@ -84,7 +96,7 @@ settle disagreements or to force a decision. ## Building the Project Locally -Iceberg is built using Gradle with Java 8, 11, or 17. +Iceberg is built using Gradle with Java 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` @@ -196,6 +208,8 @@ public interface ManageSnapshots extends PendingUpdate<Snapshot> { // existing code... // adding this method introduces an API-breaking change + // since existing classes implementing ManageSnapshots + // will no longer compile. ManageSnapshots createBranch(String name); } ``` @@ -207,8 +221,8 @@ public class SnapshotManager implements ManageSnapshots { // existing code... @Override - public ManageSnapshots createBranch(String name, long snapshotId) { - updateSnapshotReferencesOperation().createBranch(name, snapshotId); + public ManageSnapshots createBranch(String name) { + updateSnapshotReferencesOperation().createBranch(name); return this; } } diff --git a/site/docs/hive-quickstart.md b/site/docs/hive-quickstart.md index c110d56f6310..7db266d66624 100644 --- a/site/docs/hive-quickstart.md +++ b/site/docs/hive-quickstart.md @@ -18,10 +18,7 @@ title: "Hive and Iceberg Quickstart" - limitations under the License. --> - -## Hive and Iceberg Quickstart - -This guide will get you up and running with an Iceberg and Hive environment, including sample code to +This guide will get you up and running with Apache Iceberg™ using Apache Hive™, including sample code to highlight some powerful features. You can learn more about Iceberg's Hive runtime by checking out the [Hive](docs/latest/hive.md) section. - [Docker Images](#docker-images) diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index 0d2bc48464a6..6befe74b78de 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -84,7 +84,7 @@ This step can be useful to gather ongoing patches that the community thinks shou The communication can be started via a [DISCUSS] mail on the dev@ channel and the desired tickets can be added to the github milestone of the next release. -Note, creating a milestone in github requires a committer. However, a non-committer can assign tasks to a milestone if added to the list of collaborators in [.asf.yaml](https://github.com/apache/iceberg/blob/master/.asf.yaml) +Note, creating a milestone in github requires a committer. However, a non-committer can assign tasks to a milestone if added to the list of collaborators in [.asf.yaml](https://github.com/apache/iceberg/blob/main/.asf.yaml) The release status is discussed during each community sync meeting. Release manager should join the meeting to report status and discuss any release blocker. @@ -101,7 +101,7 @@ Example console output: ```text Preparing source for apache-iceberg-0.13.0-rc1 Adding version.txt and tagging release... -[master ca8bb7d0] Add version.txt for release 0.13.0 +[main ca8bb7d0] Add version.txt for release 0.13.0 1 file changed, 1 insertion(+) create mode 100644 version.txt Pushing apache-iceberg-0.13.0-rc1 to origin... diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index e4123cc579b3..ad67ba53cda3 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 (to be released) | - | - | <!-- markdown-link-check-enable --> diff --git a/site/docs/releases.md b/site/docs/releases.md index 6ba0b481fce9..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) @@ -71,6 +71,100 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ... </dependencies> ``` + +### 1.6.0 release +Apache Iceberg 1.6.0 was released on July 23, 2024. + +The 1.6.0 release contains fixes, dependency updates, and new features (like Kafak Connect commit coordinator and record converters). + +* Build + - Upgrade to Gradle 8.9 ([\#10686](https://github.com/apache/iceberg/pull/10686)) +* Core + - Add EnvironmentContext to commit summary ([\#9273](https://github.com/apache/iceberg/pull/9273)) + - Add explicit JSON parser for ConfigResponse ([\#9952](https://github.com/apache/iceberg/pull/9952)) + - Calling rewrite_position_delete_files fails on tables with more than 1k columns ([\#10020](https://github.com/apache/iceberg/pull/10020)) + - Expose table incremental scan for appends API in SerializableTable ([\#10682](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)) + - Lazily compute & cache hashCode in CharSequenceWrapper ([\#10023](https://github.com/apache/iceberg/pull/10023)) + - Pushdown data_file.content when filter manifests in entries table ([\#10203](https://github.com/apache/iceberg/pull/10203)) + - Use bulk delete when removing old metadata.json files ([\#10679](https://github.com/apache/iceberg/pull/10679)) + - JDBC Catalog: Add property to disable table initialization for JdbcCatalog ([\#10124](https://github.com/apache/iceberg/pull/10124)) + - JDBC Catalog: Exclude unexpected namespaces JdbcCatalog.listNamespaces ([\#10498](https://github.com/apache/iceberg/pull/10498)) + - JDBC Catalog: Fix JDBC Catalog table commit when migrating from schema V0 to V1 ([\#10111](https://github.com/apache/iceberg/pull/10111)) + - JDBC Catalog: Retry connections in JDBC catalog with user configured error code list ([\#10140](https://github.com/apache/iceberg/pull/10140)) + - JDBC Catalog: fix namespace SQL statement using ESCAPE character working with MySQL, PostgreSQL, ... ([\#10167](https://github.com/apache/iceberg/pull/10167)) + - REST Catalog: Assume issued_token_type is access_token to fully comply with RFC 6749 ([\#10314](https://github.com/apache/iceberg/pull/10314)) + - REST Catalog: Fix create v1 table on REST Catalog ([\#10369](https://github.com/apache/iceberg/pull/10369)) + - REST Catalog: Handles possible heap data corruption of OAuth2Util.AuthSession#headers ([\#10615](https://github.com/apache/iceberg/pull/10615)) + - REST Catalog: Handles potential NPE in RESTSessionCatalog#newSessionCache ([\#10607](https://github.com/apache/iceberg/pull/10607)) + - REST Catalog: Introduce AuthConfig ([\#10161](https://github.com/apache/iceberg/pull/10161)) + - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](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)) +* Kafka Connect + - Commit coordinator ([\#10351](https://github.com/apache/iceberg/pull/10351)) + - Record converters ([\#9641](https://github.com/apache/iceberg/pull/9641)) + - Handle namespace creation for auto table creation ([\#10186](https://github.com/apache/iceberg/pull/10186)) + - Update iceberg.hadoop-conf-dir config description ([\#10184](https://github.com/apache/iceberg/pull/10184)) +* Parquet + - Don't write column sizes when metrics mode is None ([\#10440](https://github.com/apache/iceberg/pull/10440)) +* Spark + - Fix handling of null binary values when sorting with zorder ([\#10026](https://github.com/apache/iceberg/pull/10026)) + - Spark writes/actions should only perform cleanup if failure is cleanable ([\#10373](https://github.com/apache/iceberg/pull/10373)) + - Use 'delete' if RowDelta only has delete files ([\#10123](https://github.com/apache/iceberg/pull/10123)) + - Support read of partition metadata column when table is over 1k ([\#10641](https://github.com/apache/iceberg/pull/10641)) + - Fix the setting of equalAuthorities in RemoveOrphanFilesProcedure ([\#10342](https://github.com/apache/iceberg/pull/10342)) + - Fix system function pushdown in CoW row-level commands ([\#10119](https://github.com/apache/iceberg/pull/10119)) + - Only traverse ancestors of current snapshot when building changelog scan ([\#10405](https://github.com/apache/iceberg/pull/10405)) + - Add max allowed failed commits to RewriteDataFiles when partial progress is enabled ([\#9611](https://github.com/apache/iceberg/pull/9611)) + - Fix issue when partitioning by UUID ([\#8250](https://github.com/apache/iceberg/pull/8250)) + - Use bulk deletes in rewrite manifests action ([\#10343](https://github.com/apache/iceberg/pull/10343)) +* Flink + - Remove Flink 1.16 support ([\#10154](https://github.com/apache/iceberg/pull/10154)) + - Add support for Flink 1.19 ([\#10112](https://github.com/apache/iceberg/pull/10112)) + - Apply DeleteGranularity for writes ([\#10200](https://github.com/apache/iceberg/pull/10200)) + - Move ParquetReader to LogicalTypeAnnotationVisitor ([\#9719](https://github.com/apache/iceberg/pull/9719)) + - Pre-create fieldGetters to avoid constructing them for each row ([\#10565](https://github.com/apache/iceberg/pull/10565)) + - Prevent setting endTag/endSnapshotId for streaming source ([\#10207](https://github.com/apache/iceberg/pull/10207)) + - Implement range partitioner for map data statistics ([\#9321](https://github.com/apache/iceberg/pull/9321)) + - refactor sink shuffling statistics collection ([\#10331](https://github.com/apache/iceberg/pull/10331)) +* Hive + - Fix metadata file not found ([\#10069](https://github.com/apache/iceberg/pull/10069)) + - Use base table metadata to create HiveLock ([\#10016](https://github.com/apache/iceberg/pull/10016)) + - Turn off the stats gathering when iceberg.hive.keep.stats is false ([\#10148](https://github.com/apache/iceberg/pull/10148)) +* Specs + - OpenAPI: TableRequirements should use union of subclasses ([\#10434](https://github.com/apache/iceberg/pull/10434)) + - OpenAPI: Deprecate oauth/tokens endpoint ([\#10603](https://github.com/apache/iceberg/pull/10603)) + - OpenAPI: Fix additionalProperties for SnapshotSummary ([\#9838](https://github.com/apache/iceberg/pull/9838)) + - OpenAPI: Fix property names for stats/partition stats ([\#10662](https://github.com/apache/iceberg/pull/10662)) +* Vendor Integrations + - AWS: Make sure Signer + User Agent config are both applied ([\#10198](https://github.com/apache/iceberg/pull/10198)) + - AWS: Retain Glue Catalog column comment after updating Iceberg table ([\#10276](https://github.com/apache/iceberg/pull/10276)) + - AWS: Retain Glue Catalog table description after updating Iceberg table ([\#10199](https://github.com/apache/iceberg/pull/10199)) + - AWS: Support S3 DSSE-KMS encryption ([\#8370](https://github.com/apache/iceberg/pull/8370)) + - AWS: Close underlying executor for DynamoDb LockManager ([\#10132](https://github.com/apache/iceberg/pull/10132)) + - AWS: Add Iceberg version to UserAgent in S3 requests ([\#9963](https://github.com/apache/iceberg/pull/9963)) + - Azure: Make AzureProperties w/ shared-key creds serializable ([\#10045](https://github.com/apache/iceberg/pull/10045)) +* Dependencies + - Bump Nessie to 0.92.1 + - Bump Spark 3.5 to 3.5.1 + - Bump Apache Arrow to 15.0.2 + - Bump Azure SDK to 1.2.25 + - Bump Kryo to 4.0.3 + - Bump Netty to 4.1.111.Final + - Bump Jetty to 9.4.55.v20240627 + - Bump Kafka to 3.7.1 + - Bump Apache ORC to 1.9.3 + - Bump AWS SDK to 2.26.12 + - Bump Google Cloud Libraries to 26.43.0 + +For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.6.0). + +## Past releases + ### 1.5.2 release Apache Iceberg 1.5.2 was released on May 9, 2024. @@ -161,8 +255,6 @@ The 1.5.0 release adds a variety of new features and bug fixes. For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.5.0). -## Past releases - ### 1.4.3 Release Apache Iceberg 1.4.3 was released on December 27, 2023. The main issue it solves is missing files from a transaction retry with conflicting manifests. It is recommended to upgrade if you use transactions. diff --git a/site/docs/spark-quickstart.md b/site/docs/spark-quickstart.md index 5a940009f9a3..e98bedb49825 100644 --- a/site/docs/spark-quickstart.md +++ b/site/docs/spark-quickstart.md @@ -18,9 +18,7 @@ title: "Spark and Iceberg Quickstart" - limitations under the License. --> -## Spark and Iceberg Quickstart - -This guide will get you up and running with an Iceberg and Spark environment, including sample code to +This guide will get you up and running with Apache Iceberg™ using Apache Spark™, including sample code to highlight some powerful features. You can learn more about Iceberg's Spark runtime by checking out the [Spark](docs/latest/spark-ddl.md) section. - [Docker-Compose](#docker-compose) diff --git a/site/docs/vendors.md b/site/docs/vendors.md index dc20783da5ba..dc8cf135046c 100644 --- a/site/docs/vendors.md +++ b/site/docs/vendors.md @@ -22,6 +22,13 @@ title: "Vendors" This page contains some of the vendors who are shipping and supporting Apache Iceberg in their products +### [Bodo](https://bodo.ai) + +Bodo is a high performance SQL & Python compute engine that brings HPC and supercomputing techniques to data analytics. +Bodo supports Apache Iceberg tables as a first-class table format and storage, enabling users to read and write Iceberg +tables with Bodo's high-performance data processing engine. Bodo is available as a cloud service on +AWS and Azure, and as well as an on-premises solution. + ### [CelerData](https://celerdata.com) CelerData provides commercial offerings for [StarRocks](https://www.starrocks.io/), a distributed MPP SQL engine for enterprise analytics on Iceberg. With its fully vectorized technology, local caching, and intelligent materialized view, StarRocks delivers sub-second query latency for both batch and real-time analytics. CelerData offers both an [enterprise deployment](https://celerdata.com/celerdata-enterprise) and a [cloud service](https://celerdata.com/celerdata-cloud) to help customers use StarRocks more smoothly. Learn more about how to query Iceberg with StarRocks [here](https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog). diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 0642281a7c24..db9bafb00f05 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -17,7 +17,7 @@ INHERIT: ./nav.yml -site_name: Apache Iceberg +site_name: "Apache Iceberg™" theme: custom_dir: overrides @@ -78,8 +78,8 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.5.2' - nessieVersion: '0.77.1' + icebergVersion: '1.6.1' + nessieVersion: '0.92.1' flinkVersion: '1.19.0' flinkVersionMajor: '1.19' social: diff --git a/site/nav.yml b/site/nav.yml index 3fe17a94953f..cad34766be4c 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -23,6 +23,8 @@ nav: - Docs: - nightly: '!include docs/docs/nightly/mkdocs.yml' - latest: '!include docs/docs/latest/mkdocs.yml' + - 1.6.1: '!include docs/docs/1.6.1/mkdocs.yml' + - 1.6.0: '!include docs/docs/1.6.0/mkdocs.yml' - 1.5.2: '!include docs/docs/1.5.2/mkdocs.yml' - 1.5.1: '!include docs/docs/1.5.1/mkdocs.yml' - 1.5.0: '!include docs/docs/1.5.0/mkdocs.yml' diff --git a/site/overrides/home.html b/site/overrides/home.html index f6479d46ea9b..65d971e0a134 100644 --- a/site/overrides/home.html +++ b/site/overrides/home.html @@ -34,7 +34,7 @@ <div class="d-flex flex-row"> <div class="col-lg-12"> <div class="intro-message"> - <h1>Apache Iceberg</h1> + <h1>Apache Iceberg™</h1> <h3>The open table format for analytic datasets.</h3> <hr class="intro-divider" /> <ul class="list-inline intro-social-buttons"> @@ -61,7 +61,7 @@ <h3>The open table format for analytic datasets.</h3> <div class="col-lg-2"></div> <div class="col-lg-8"> <br /><br /> - <h2>What is Iceberg?</h2> + <h2>What is Apache Iceberg™?</h2> <br /> <p> diff --git a/site/overrides/partials/header.html b/site/overrides/partials/header.html index 6d943e0c0888..62b02698b62e 100644 --- a/site/overrides/partials/header.html +++ b/site/overrides/partials/header.html @@ -50,13 +50,10 @@ <div class="md-header__title" data-md-component="header-title"> <div class="md-header__ellipsis"> <div class="md-header__topic"> - <span class="md-ellipsis"> </span> + <span class="md-ellipsis">{{ config.site_name }}</span> </div> <div class="md-header__topic" data-md-component="header-topic"> - <span class="md-ellipsis"> - {% if page.meta and page.meta.title %} {{ page.meta.title }} {% else - %} {{ page.title }} {% endif %} - </span> + <span class="md-ellipsis">{{ config.site_name }}</span> </div> </div> </div> diff --git a/site/requirements.txt b/site/requirements.txt index 75beac22648f..377fdc43fef2 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -15,9 +15,9 @@ # specific language governing permissions and limitations # under the License. -mkdocs-awesome-pages-plugin==2.9.2 +mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.28 +mkdocs-material==9.5.34 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.1 diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 63c1a8e90a2e..c081cffeb644 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -59,6 +59,10 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } compileOnly libs.errorprone.annotations compileOnly libs.avro.avro @@ -137,6 +141,10 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer dependencies { implementation "org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}" + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } implementation libs.roaringbitmap compileOnly "org.scala-lang:scala-library" diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index 3f56d82d9b70..7741ae39d620 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -48,6 +48,7 @@ import org.apache.spark.sql.internal.SQLConf; import org.junit.Assert; import org.junit.Assume; +import org.junit.Ignore; import org.junit.Test; public class TestCopyOnWriteMerge extends TestMerge { @@ -70,6 +71,7 @@ protected Map<String, String> extraTableProperties() { } @Test + @Ignore // Ignored due to https://github.com/apache/iceberg/issues/10040 public synchronized void testMergeWithConcurrentTableRefresh() throws Exception { // this test can only be run with Hive tables as it requires a reliable lock // also, the table cache must be enabled so that the same table instance can be reused diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index ef3138d677c6..afb0f434aa41 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -20,6 +20,9 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.BaseTable; @@ -47,6 +50,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkAggregates; import org.apache.iceberg.spark.SparkFilters; @@ -362,15 +366,54 @@ public void pruneColumns(StructType requestedSchema) { private Schema schemaWithMetadataColumns() { // metadata columns - List<Types.NestedField> fields = + List<Types.NestedField> metadataFields = metaColumns.stream() .distinct() .map(name -> MetadataColumns.metadataColumn(table, name)) .collect(Collectors.toList()); - Schema meta = new Schema(fields); + Schema metadataSchema = calculateMetadataSchema(metadataFields); // schema or rows returned by readers - return TypeUtil.join(schema, meta); + return TypeUtil.join(schema, metadataSchema); + } + + private Schema calculateMetadataSchema(List<Types.NestedField> metaColumnFields) { + Optional<Types.NestedField> partitionField = + metaColumnFields.stream() + .filter(f -> MetadataColumns.PARTITION_COLUMN_ID == f.fieldId()) + .findFirst(); + + // only calculate potential column id collision if partition metadata column was requested + if (!partitionField.isPresent()) { + return new Schema(metaColumnFields); + } + + Set<Integer> idsToReassign = + TypeUtil.indexById(partitionField.get().type().asStructType()).keySet(); + + // Calculate used ids by union metadata columns with all base table schemas + Set<Integer> currentlyUsedIds = + metaColumnFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + Set<Integer> allUsedIds = + table.schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Reassign selected ids to deduplicate with used ids. + AtomicInteger nextId = new AtomicInteger(); + return new Schema( + metaColumnFields, + table.schema().identifierFieldIds(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); } @Override diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index e24e74383bc8..127b0eb66fe9 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_BATCH_SIZE; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.spark.sql.functions.expr; import static org.apache.spark.sql.functions.lit; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -31,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; @@ -53,6 +55,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.types.StructType; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -169,6 +172,52 @@ public void testSpecAndPartitionMetadataColumns() { sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); } + @Test + public void testPartitionMetadataColumnWithManyColumns() { + // TODO: support metadata structs in vectorized ORC reads + Assume.assumeFalse(fileFormat == FileFormat.ORC && vectorized); + List<Types.NestedField> fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List<Types.NestedField> additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema manyColumnsSchema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(manyColumnsSchema).identity("id").build(); + + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit( + base, + base.updateSchema(manyColumnsSchema, manyColumnsSchema.highestFieldId()) + .updatePartitionSpec(spec)); + + Dataset<Row> df = + spark + .range(2) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(TABLE_NAME); + + Assert.assertEquals(2, spark.table(TABLE_NAME).select("*", "_partition").count()); + List<Object[]> expected = + ImmutableList.of(row(row(0L), 0L, "0", "0", "0"), row(row(1L), 1L, "1", "1", "1")); + assertEquals( + "Rows must match", + expected, + sql("SELECT _partition, id, c999, c1000, c1001 FROM %s ORDER BY id", TABLE_NAME)); + } + @Test public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { Assume.assumeTrue(fileFormat == FileFormat.PARQUET); diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 9c584ad3a21b..3b1761d39f63 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -59,6 +59,10 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } compileOnly libs.errorprone.annotations compileOnly libs.avro.avro @@ -138,6 +142,10 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer dependencies { implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } implementation libs.roaringbitmap compileOnly "org.scala-lang:scala-library" diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 6b97e48133fd..9dc214a755d3 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -20,6 +20,9 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.BaseTable; @@ -48,6 +51,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkAggregates; import org.apache.iceberg.spark.SparkReadConf; @@ -342,15 +346,54 @@ public void pruneColumns(StructType requestedSchema) { private Schema schemaWithMetadataColumns() { // metadata columns - List<Types.NestedField> fields = + List<Types.NestedField> metadataFields = metaColumns.stream() .distinct() .map(name -> MetadataColumns.metadataColumn(table, name)) .collect(Collectors.toList()); - Schema meta = new Schema(fields); + Schema metadataSchema = calculateMetadataSchema(metadataFields); // schema or rows returned by readers - return TypeUtil.join(schema, meta); + return TypeUtil.join(schema, metadataSchema); + } + + private Schema calculateMetadataSchema(List<Types.NestedField> metaColumnFields) { + Optional<Types.NestedField> partitionField = + metaColumnFields.stream() + .filter(f -> MetadataColumns.PARTITION_COLUMN_ID == f.fieldId()) + .findFirst(); + + // only calculate potential column id collision if partition metadata column was requested + if (!partitionField.isPresent()) { + return new Schema(metaColumnFields); + } + + Set<Integer> idsToReassign = + TypeUtil.indexById(partitionField.get().type().asStructType()).keySet(); + + // Calculate used ids by union metadata columns with all base table schemas + Set<Integer> currentlyUsedIds = + metaColumnFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + Set<Integer> allUsedIds = + table.schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Reassign selected ids to deduplicate with used ids. + AtomicInteger nextId = new AtomicInteger(); + return new Schema( + metaColumnFields, + table.schema().identifierFieldIds(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); } @Override diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 778c46bba6b6..0ba34a638a63 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_BATCH_SIZE; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.spark.sql.functions.expr; import static org.apache.spark.sql.functions.lit; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -31,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; @@ -53,6 +55,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.types.StructType; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -169,6 +172,50 @@ public void testSpecAndPartitionMetadataColumns() { sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); } + @Test + public void testPartitionMetadataColumnWithManyColumns() { + List<Types.NestedField> fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List<Types.NestedField> additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema manyColumnsSchema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(manyColumnsSchema).identity("id").build(); + + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit( + base, + base.updateSchema(manyColumnsSchema, manyColumnsSchema.highestFieldId()) + .updatePartitionSpec(spec)); + + Dataset<Row> df = + spark + .range(2) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(TABLE_NAME); + + Assert.assertEquals(2, spark.table(TABLE_NAME).select("*", "_partition").count()); + List<Object[]> expected = + ImmutableList.of(row(row(0L), 0L, "0", "0", "0"), row(row(1L), 1L, "1", "1", "1")); + assertEquals( + "Rows must match", + expected, + sql("SELECT _partition, id, c999, c1000, c1001 FROM %s ORDER BY id", TABLE_NAME)); + } + @Test public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { Assume.assumeTrue(fileFormat == FileFormat.PARQUET); diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index f1af2120ef8c..c8d8bbf396a0 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -59,6 +59,11 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation("org.apache.datasketches:datasketches-java:${libs.versions.datasketches.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } compileOnly libs.errorprone.annotations compileOnly libs.avro.avro @@ -137,6 +142,10 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer dependencies { implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } implementation libs.roaringbitmap compileOnly "org.scala-lang:scala-library" @@ -284,6 +293,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap' + relocate 'org.apache.datasketches', 'org.apache.iceberg.shaded.org.apache.datasketches' archiveClassifier.set(null) } diff --git a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java index e2ce5e956348..ed97e6b08414 100644 --- a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java +++ b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -20,49 +20,41 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; -import static org.apache.spark.sql.functions.lit; import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AppendFiles; import org.apache.iceberg.BatchScan; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.FileGenerationUtil; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.ScanTask; -import org.apache.iceberg.Schema; import org.apache.iceberg.SparkDistributedDataScan; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; -import org.apache.spark.sql.types.StructType; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -108,10 +100,8 @@ public class PlanningBenchmark { Expressions.and(PARTITION_PREDICATE, SORT_KEY_PREDICATE); private static final int NUM_PARTITIONS = 30; - private static final int NUM_REAL_DATA_FILES_PER_PARTITION = 25; - private static final int NUM_REPLICA_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; private static final int NUM_DELETE_FILES_PER_PARTITION = 50; - private static final int NUM_ROWS_PER_DATA_FILE = 500; private final Configuration hadoopConf = new Configuration(); private SparkSession spark; @@ -285,99 +275,43 @@ private void dropTable() { sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); } - private DataFile loadAddedDataFile() { - table.refresh(); - - Iterable<DataFile> dataFiles = table.currentSnapshot().addedDataFiles(table.io()); - return Iterables.getOnlyElement(dataFiles); - } - - private DeleteFile loadAddedDeleteFile() { - table.refresh(); - - Iterable<DeleteFile> deleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); - return Iterables.getOnlyElement(deleteFiles); - } - - private void initDataAndDeletes() throws NoSuchTableException { - Schema schema = table.schema(); - PartitionSpec spec = table.spec(); - LocationProvider locations = table.locationProvider(); - + private void initDataAndDeletes() { for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { - Dataset<Row> inputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)) - .drop(SORT_KEY_COLUMN) - .withColumn(SORT_KEY_COLUMN, lit(Integer.MIN_VALUE)); - - for (int fileOrdinal = 0; fileOrdinal < NUM_REAL_DATA_FILES_PER_PARTITION; fileOrdinal++) { - appendAsFile(inputDF); - } + StructLike partition = TestHelpers.Row.of(partitionOrdinal); - DataFile dataFile = loadAddedDataFile(); - - sql( - "DELETE FROM %s WHERE ss_item_sk IS NULL AND %s = %d", - TABLE_NAME, PARTITION_COLUMN, partitionOrdinal); - - DeleteFile deleteFile = loadAddedDeleteFile(); - - AppendFiles append = table.newFastAppend(); + RowDelta rowDelta = table.newRowDelta(); - for (int fileOrdinal = 0; fileOrdinal < NUM_REPLICA_DATA_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DataFile replicaDataFile = - DataFiles.builder(spec) - .copy(dataFile) - .withPath(locations.newDataLocation(spec, dataFile.partition(), replicaFileName)) - .build(); - append.appendFile(replicaDataFile); + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + rowDelta.addRows(dataFile); } - append.commit(); - - RowDelta rowDelta = table.newRowDelta(); + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DeleteFile replicaDeleteFile = - FileMetadata.deleteFileBuilder(spec) - .copy(deleteFile) - .withPath(locations.newDataLocation(spec, deleteFile.partition(), replicaFileName)) - .build(); - rowDelta.addDeletes(replicaDeleteFile); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); } rowDelta.commit(); - - Dataset<Row> sortedInputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(SORT_KEY_COLUMN) - .withColumn(SORT_KEY_COLUMN, lit(SORT_KEY_VALUE)) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); - appendAsFile(sortedInputDF); } } - private void appendAsFile(Dataset<Row> df) throws NoSuchTableException { - df.coalesce(1).writeTo(TABLE_NAME).append(); + private DataFile generateDataFile(StructLike partition, int sortKeyMin, int sortKeyMax) { + int sortKeyFieldId = table.schema().findField(SORT_KEY_COLUMN).fieldId(); + ByteBuffer lower = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMin); + Map<Integer, ByteBuffer> lowerBounds = ImmutableMap.of(sortKeyFieldId, lower); + ByteBuffer upper = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMax); + Map<Integer, ByteBuffer> upperBounds = ImmutableMap.of(sortKeyFieldId, upper); + return FileGenerationUtil.generateDataFile(table, partition, lowerBounds, upperBounds); } private String newWarehouseDir() { return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); } - private Dataset<Row> randomDataDF(Schema schema, int numRows) { - Iterable<InternalRow> rows = RandomData.generateSpark(schema, numRows, 0); - JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD<InternalRow> rowRDD = context.parallelize(Lists.newArrayList(rows)); - StructType rowSparkType = SparkSchemaUtil.convert(schema); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); - } - private List<ScanTask> planFilesWithoutColumnStats(BatchScan scan, Expression predicate) { return planFiles(scan, predicate, false); } 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/SparkTableUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index 7a96e97fb98a..e103104171f6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -851,6 +851,12 @@ private static void deleteManifests(FileIO io, List<ManifestFile> manifests) { .run(item -> io.deleteFile(item.path())); } + public static Dataset<Row> loadTable(SparkSession spark, Table table, long snapshotId) { + SparkTable sparkTable = new SparkTable(table, snapshotId, false); + DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); + return Dataset.ofRows(spark, relation); + } + public static Dataset<Row> loadMetadataTable( SparkSession spark, Table table, MetadataTableType type) { return loadMetadataTable(spark, table, type, ImmutableMap.of()); @@ -858,11 +864,16 @@ public static Dataset<Row> loadMetadataTable( public static Dataset<Row> loadMetadataTable( SparkSession spark, Table table, MetadataTableType type, Map<String, String> extraOptions) { - SparkTable metadataTable = - new SparkTable(MetadataTableUtils.createMetadataTableInstance(table, type), false); + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); + return Dataset.ofRows(spark, relation); + } + + private static DataSourceV2Relation createRelation( + SparkTable sparkTable, Map<String, String> extraOptions) { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); - return Dataset.ofRows( - spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); + return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java new file mode 100644 index 000000000000..a508021c1040 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ImmutableComputeTableStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Computes the statistics of the given columns and stores it as Puffin files. */ +public class ComputeTableStatsSparkAction extends BaseSparkAction<ComputeTableStatsSparkAction> + implements ComputeTableStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputeTableStatsSparkAction.class); + private static final Result EMPTY_RESULT = ImmutableComputeTableStats.Result.builder().build(); + + private final Table table; + private List<String> columns; + private Snapshot snapshot; + + ComputeTableStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputeTableStatsSparkAction self() { + return this; + } + + @Override + public ComputeTableStats columns(String... newColumns) { + Preconditions.checkArgument( + newColumns != null && newColumns.length > 0, "Columns cannot be null/empty"); + this.columns = ImmutableList.copyOf(ImmutableSet.copyOf(newColumns)); + return this; + } + + @Override + public ComputeTableStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute stats for table {}", table.name()); + return EMPTY_RESULT; + } + validateColumns(); + JobGroupInfo info = newJobGroupInfo("COMPUTE-TABLE-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info( + "Computing stats for columns {} in {} (snapshot {})", + columns(), + table.name(), + snapshotId()); + List<Blob> blobs = generateNDVBlobs(); + StatisticsFile statisticsFile = writeStatsFile(blobs); + table.updateStatistics().setStatistics(snapshotId(), statisticsFile).commit(); + return ImmutableComputeTableStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private StatisticsFile writeStatsFile(List<Blob> blobs) { + LOG.info("Writing stats for table {} for snapshot {}", table.name(), snapshotId()); + OutputFile outputFile = table.io().newOutputFile(outputPath()); + try (PuffinWriter writer = Puffin.write(outputFile).createdBy(appIdentifier()).build()) { + blobs.forEach(writer::add); + writer.finish(); + return new GenericStatisticsFile( + snapshotId(), + outputFile.location(), + writer.fileSize(), + writer.footerSize(), + GenericBlobMetadata.from(writer.writtenBlobsMetadata())); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private List<Blob> generateNDVBlobs() { + return NDVSketchUtil.generateBlobs(spark(), table, snapshot, columns()); + } + + private List<String> columns() { + if (columns == null) { + Schema schema = table.schemas().get(snapshot.schemaId()); + this.columns = + schema.columns().stream() + .filter(nestedField -> nestedField.type().isPrimitiveType()) + .map(Types.NestedField::name) + .collect(Collectors.toList()); + } + return columns; + } + + private void validateColumns() { + Schema schema = table.schemas().get(snapshot.schemaId()); + Preconditions.checkArgument(!columns().isEmpty(), "No columns found to compute stats"); + for (String columnName : columns()) { + Types.NestedField field = schema.findField(columnName); + Preconditions.checkArgument(field != null, "Can't find column %s in %s", columnName, schema); + Preconditions.checkArgument( + field.type().isPrimitiveType(), + "Can't compute stats on non-primitive type column: %s (%s)", + columnName, + field.type()); + } + } + + private String appIdentifier() { + String icebergVersion = IcebergBuild.fullVersion(); + String sparkVersion = spark().version(); + return String.format("Iceberg %s Spark %s", icebergVersion, sparkVersion); + } + + private long snapshotId() { + return snapshot.snapshotId(); + } + + private String jobDesc() { + return String.format( + "Computing table stats for %s (snapshot_id=%s, columns=%s)", + table.name(), snapshotId(), columns()); + } + + private String outputPath() { + TableOperations operations = ((HasTableOperations) table).operations(); + String fileName = String.format("%s-%s.stats", snapshotId(), UUID.randomUUID()); + return operations.metadataFileLocation(fileName); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java new file mode 100644 index 000000000000..c8a20d3cca73 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.theta.CompactSketch; +import org.apache.datasketches.theta.Sketch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.PuffinCompressionCodec; +import org.apache.iceberg.puffin.StandardBlobTypes; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.stats.ThetaSketchAgg; + +public class NDVSketchUtil { + + private NDVSketchUtil() {} + + public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv"; + + static List<Blob> generateBlobs( + SparkSession spark, Table table, Snapshot snapshot, List<String> columns) { + Row sketches = computeNDVSketches(spark, table, snapshot, columns); + Schema schema = table.schemas().get(snapshot.schemaId()); + List<Blob> blobs = Lists.newArrayList(); + for (int i = 0; i < columns.size(); i++) { + Types.NestedField field = schema.findField(columns.get(i)); + Sketch sketch = CompactSketch.wrap(Memory.wrap((byte[]) sketches.get(i))); + blobs.add(toBlob(field, sketch, snapshot)); + } + return blobs; + } + + private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snapshot) { + return new Blob( + StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + ByteBuffer.wrap(sketch.toByteArray()), + PuffinCompressionCodec.ZSTD, + ImmutableMap.of( + APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, + String.valueOf((long) sketch.getEstimate()))); + } + + private static Row computeNDVSketches( + SparkSession spark, Table table, Snapshot snapshot, List<String> colNames) { + Dataset<Row> inputDF = SparkTableUtil.loadTable(spark, table, snapshot.snapshotId()); + return inputDF.select(toAggColumns(colNames)).first(); + } + + private static Column[] toAggColumns(List<String> colNames) { + return colNames.stream().map(NDVSketchUtil::toAggColumn).toArray(Column[]::new); + } + + private static Column toAggColumn(String colName) { + ThetaSketchAgg agg = new ThetaSketchAgg(colName); + return new Column(agg.toAggregateExpression()); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index fb67ded96e35..f845386d30c4 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -20,6 +20,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.ComputeTableStats; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -96,4 +97,9 @@ public DeleteReachableFilesSparkAction deleteReachableFiles(String metadataLocat public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) { return new RewritePositionDeleteFilesSparkAction(spark, table); } + + @Override + public ComputeTableStats computeTableStats(Table table) { + return new ComputeTableStatsSparkAction(spark, table); + } } diff --git a/spark/v3.5/spark/src/main/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<Object> min; + private final Optional<Object> max; + private final OptionalLong nullCount; + private final OptionalLong avgLen; + private final OptionalLong maxLen; + private final Optional<Histogram> 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<Object> min() { + return min; + } + + @Override + public Optional<Object> 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> 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<NamedReference, ColumnStatistics> colStatsMap = Collections.emptyMap(); + if (readConf.reportColumnStats() && cboEnabled) { + colStatsMap = Maps.newHashMap(); + List<StatisticsFile> files = table.statisticsFiles(); + if (!files.isEmpty()) { + List<BlobMetadata> 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<NamedReference, ColumnStatistics> colstats; - Stats(long sizeInBytes, long numRows) { + Stats(long sizeInBytes, long numRows, Map<NamedReference, ColumnStatistics> 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<NamedReference, ColumnStatistics> columnStats() { + return colstats; + } } diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala new file mode 100644 index 000000000000..cca16960f434 --- /dev/null +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.stats + +import java.nio.ByteBuffer +import org.apache.datasketches.common.Family +import org.apache.datasketches.memory.Memory +import org.apache.datasketches.theta.CompactSketch +import org.apache.datasketches.theta.SetOperationBuilder +import org.apache.datasketches.theta.Sketch +import org.apache.datasketches.theta.UpdateSketch +import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.Conversions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.UTF8String + +/** + * ThetaSketchAgg generates Alpha family sketch with default seed. + * The values fed to the sketch are converted to bytes using Iceberg's single value serialization. + * The result returned is an array of bytes of Compact Theta sketch of Datasketches library, + * which should be deserialized to Compact sketch before using. + * + * See [[https://iceberg.apache.org/puffin-spec/]] for more information. + * + */ +case class ThetaSketchAgg( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + + private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) + + def this(colName: String) = { + this(col(colName).expr, 0, 0) + } + + override def dataType: DataType = BinaryType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): Sketch = { + UpdateSketch.builder.setFamily(Family.ALPHA).build() + } + + override def update(buffer: Sketch, input: InternalRow): Sketch = { + val value = child.eval(input) + if (value != null) { + val icebergValue = toIcebergValue(value) + val byteBuffer = Conversions.toByteBuffer(icebergType, icebergValue) + buffer.asInstanceOf[UpdateSketch].update(byteBuffer) + } + buffer + } + + private def toIcebergValue(value: Any): Any = { + value match { + case s: UTF8String => s.toString + case d: Decimal => d.toJavaBigDecimal + case b: Array[Byte] => ByteBuffer.wrap(b) + case _ => value + } + } + + override def merge(buffer: Sketch, input: Sketch): Sketch = { + new SetOperationBuilder().buildUnion.union(buffer, input) + } + + override def eval(buffer: Sketch): Any = { + toBytes(buffer) + } + + override def serialize(buffer: Sketch): Array[Byte] = { + toBytes(buffer) + } + + override def deserialize(storageFormat: Array[Byte]): Sketch = { + CompactSketch.wrap(Memory.wrap(storageFormat)) + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + + private def toBytes(sketch: Sketch): Array[Byte] = { + val compactSketch = sketch.compact() + compactSketch.toByteArray + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java new file mode 100644 index 000000000000..4cab0d42815c --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -0,0 +1,431 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import static org.apache.iceberg.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestTemplate; + +public class TestComputeTableStatsAction extends CatalogTestBase { + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema SCHEMA_WITH_NESTED_COLUMN = + new Schema( + required(4, "nestedStructCol", NESTED_STRUCT_TYPE), + required(5, "stringCol", Types.StringType.get())); + + @TestTemplate + public void testLoadingTableDirectly() { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("INSERT into %s values(1, 'abcd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + StatisticsFile statisticsFile = results.statisticsFile(); + Assertions.assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 2); + } + + @TestTemplate + public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + // To create multiple splits on the mapper + table + .updateProperties() + .set("read.split.target-size", "100") + .set("write.parquet.row-group-size-bytes", "100") + .commit(); + List<SimpleRecord> records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns("id", "data").execute(); + assertNotNull(results); + + List<StatisticsFile> statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 2); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertEquals( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), + String.valueOf(4)); + } + + @TestTemplate + public void testComputeTableStatsActionWithoutExplicitColumns() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List<SimpleRecord> records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + assertNotNull(results); + + Assertions.assertEquals(1, table.statisticsFiles().size()); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + Assertions.assertEquals(2, statisticsFile.blobMetadata().size()); + assertNotEquals(0, statisticsFile.fileSizeInBytes()); + Assertions.assertEquals( + 4, + Long.parseLong( + statisticsFile + .blobMetadata() + .get(0) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))); + Assertions.assertEquals( + 4, + Long.parseLong( + statisticsFile + .blobMetadata() + .get(1) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))); + } + + @TestTemplate + public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> actions.computeTableStats(table).columns("id1").execute()); + String message = exception.getMessage(); + assertTrue(message.contains("Can't find column id1 in table")); + } + + @TestTemplate + public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); + Assertions.assertNull(result.statisticsFile()); + } + + @TestTemplate + public void testComputeTableStatsWithNullValues() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + List<SimpleRecord> records = + Lists.newArrayList( + new SimpleRecord(1, null), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + assertNotNull(results); + + List<StatisticsFile> statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertEquals( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), + String.valueOf(4)); + } + + @TestTemplate + public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() + throws NoSuchTableException, ParseException { + SparkActions actions = SparkActions.get(); + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + long snapshotId1 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + // Snapshot id not specified + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + assertDoesNotThrow(() -> actions.computeTableStats(table).columns("data").execute()); + + sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); + // Append data to create snapshot + sql("INSERT into %s values(1)", tableName); + table.refresh(); + long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + + // Snapshot id specified + assertDoesNotThrow( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()); + + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()); + String message = exception.getMessage(); + assertTrue(message.contains("Can't find column data in table")); + } + + @TestTemplate + public void testComputeTableStatsWhenSnapshotIdNotSpecified() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + + assertNotNull(results); + + List<StatisticsFile> statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertEquals( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), + String.valueOf(1)); + } + + @TestTemplate + public void testComputeTableStatsWithNestedSchema() + throws NoSuchTableException, ParseException, IOException { + List<Record> records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, + SCHEMA_WITH_NESTED_COLUMN, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + Table tbl = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + actions.computeTableStats(tbl).execute(); + + tbl.refresh(); + List<StatisticsFile> statisticsFiles = tbl.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + } + + @TestTemplate + public void testComputeTableStatsWithNoComputableColumns() throws IOException { + List<Record> records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, NESTED_SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + table.refresh(); + SparkActions actions = SparkActions.get(); + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, () -> actions.computeTableStats(table).execute()); + Assertions.assertEquals(exception.getMessage(), "No columns found to compute stats"); + } + + @TestTemplate + public void testComputeTableStatsOnByteColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("byte_col", "TINYINT"); + } + + @TestTemplate + public void testComputeTableStatsOnShortColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("short_col", "SMALLINT"); + } + + @TestTemplate + public void testComputeTableStatsOnIntColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("int_col", "INT"); + } + + @TestTemplate + public void testComputeTableStatsOnLongColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("long_col", "BIGINT"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampNtzColumn() + throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP_NTZ"); + } + + @TestTemplate + public void testComputeTableStatsOnDateColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("date_col", "DATE"); + } + + @TestTemplate + public void testComputeTableStatsOnDecimalColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("decimal_col", "DECIMAL(20, 2)"); + } + + @TestTemplate + public void testComputeTableStatsOnBinaryColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("binary_col", "BINARY"); + } + + public void testComputeTableStats(String columnName, String type) + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, %s %s) USING iceberg", tableName, columnName, type); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + Dataset<Row> dataDF = randomDataDF(table.schema()); + append(tableName, dataDF); + + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns(columnName).execute(); + assertNotNull(results); + + List<StatisticsFile> statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertNotNull( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)); + } + + private GenericRecord createNestedRecord() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_NESTED_COLUMN); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, 0L); + leaf.set(1, 0.0); + nested.set(0, leaf); + record.set(0, nested); + record.set(1, "data"); + return record; + } + + private Dataset<Row> randomDataDF(Schema schema) { + Iterable<InternalRow> rows = RandomData.generateSpark(schema, 10, 0); + JavaRDD<InternalRow> rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset<Row> df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index a0f1fba0434c..e2ebf7e8817c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -186,6 +186,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @TestTemplate @@ -746,6 +751,8 @@ public void testStructOfThreeLevelLists() throws Exception { @TestTemplate public void testTwoLevelList() throws IOException { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -829,6 +836,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -851,6 +860,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -876,6 +887,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -903,6 +916,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); 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<SimpleRecord> 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<String, String> reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map<String, String> 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<SimpleRecord> 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<String, String> reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map<String, String> 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<String, Long> 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<SimpleRecord> 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<String, String> reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map<String, String> 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<String, Long> 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<NamedReference, ColumnStatistics> columnStats = stats.columnStats(); + assertThat(columnStats.isEmpty()); + } + + private void checkColStatisticsReported( + SparkScan scan, long expectedRowCount, Map<String, Long> expectedNDVs) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map<NamedReference, ColumnStatistics> columnStats = stats.columnStats(); + for (Map.Entry<String, Long> entry : expectedNDVs.entrySet()) { + assertThat(columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } + } + private static LiteralValue<Integer> intLit(int value) { return LiteralValue.apply(value, DataTypes.IntegerType); } 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 } }