diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index fd7f6f14c2ef..245da4ecd4d1 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -46,7 +46,7 @@ jobs: # # See https://github.com/actions/checkout/issues/124 fetch-depth: 0 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 67f5acdb32b2..6fd97e662ab0 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -62,7 +62,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -74,7 +74,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v3 if: failure() with: @@ -91,7 +91,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -103,7 +103,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.3 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v3 if: failure() with: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 4ea17af6c0cb..702ae9bc898d 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -60,12 +60,12 @@ jobs: strategy: matrix: jvm: [8, 11] - flink: ['1.15', '1.16', '1.17'] + flink: ['1.16', '1.17', '1.18'] env: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index d7474a185436..f582e516fcd1 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -60,7 +60,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -86,7 +86,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 4c8e3a5b8d92..4936e2b6514b 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -57,7 +57,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -81,7 +81,7 @@ jobs: runs-on: ubuntu-22.04 steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 @@ -91,7 +91,7 @@ jobs: runs-on: ubuntu-22.04 steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index f2d6c659f61e..e76590543852 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -79,7 +79,7 @@ jobs: with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/open-api.yml b/.github/workflows/open-api.yml index ae18c646e591..397d1bf30e4a 100644 --- a/.github/workflows/open-api.yml +++ b/.github/workflows/open-api.yml @@ -40,7 +40,7 @@ jobs: steps: - uses: actions/checkout@v4 - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: 3.9 - name: Install diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 92c6b2b09223..c2809cfdcbac 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -34,7 +34,7 @@ jobs: with: # we need to fetch all tags so that getProjectVersion() in build.gradle correctly determines the next SNAPSHOT version from the newest tag fetch-depth: 0 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index e1f750041890..434b79755069 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -49,7 +49,7 @@ jobs: with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index a05c9aa19e29..3dacdf77c1a8 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -63,7 +63,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -93,7 +93,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -123,7 +123,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 17 diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 566ae2441ea0..3c98621c2beb 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -31,7 +31,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-22.04 steps: - - uses: actions/stale@v8.0.0 + - uses: actions/stale@v9.0.0 with: stale-issue-label: 'stale' exempt-issue-labels: 'not-stale' diff --git a/api/src/main/java/org/apache/iceberg/StreamingUpdate.java b/api/src/main/java/org/apache/iceberg/StreamingUpdate.java new file mode 100644 index 000000000000..01f2f4e79a1f --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/StreamingUpdate.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 org.apache.iceberg.exceptions.ValidationException; + +/** + * API for appending sequential updates to a table + * + *

This API accumulates batches of file additions and deletions by order, produces a new {@link + * Snapshot} of the changes where each batch is added to a new data sequence number, and commits + * that snapshot as the current. + * + *

When committing, these changes will be applied to the latest table snapshot. Commit conflicts + * will be resolved by applying the changes to the new latest snapshot and reattempting the commit. + * If any of the deleted files are no longer in the latest snapshot when reattempting, the commit + * will throw a {@link ValidationException}. + */ +public interface StreamingUpdate extends SnapshotUpdate { + /** + * Remove a data file from the current table state. + * + *

This rewrite operation may change the size or layout of the data files. When applicable, it + * is also recommended to discard already deleted records while rewriting data files. However, the + * set of live data records must never change. + * + * @param dataFile a rewritten data file + * @return this for method chaining + */ + default StreamingUpdate deleteFile(DataFile dataFile) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement deleteFile"); + } + + /** + * Remove a delete file from the table state. + * + *

This rewrite operation may change the size or layout of the delete files. When applicable, + * it is also recommended to discard delete records for files that are no longer part of the table + * state. However, the set of applicable delete records must never change. + * + * @param deleteFile a rewritten delete file + * @return this for method chaining + */ + default StreamingUpdate deleteFile(DeleteFile deleteFile) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement deleteFile"); + } + + /** + * Add a new data file to a specific. All files in this batch will receive the same data sequence + * number. + * + *

This rewrite operation may change the size or layout of the data files. When applicable, it + * is also recommended to discard already deleted records while rewriting data files. However, the + * set of live data records must never change. + * + * @param dataFile a new data file + * @param batchOrdinal The batch ordinal to associate with this data file + * @return this for method chaining + */ + default StreamingUpdate addFile(DataFile dataFile, int batchOrdinal) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement addFile"); + } + + /** + * Add a new delete file to a specific batch. All files in this batch will receive the same data + * sequence number. + * + *

This rewrite operation may change the size or layout of the delete files. When applicable, + * it is also recommended to discard delete records for files that are no longer part of the table + * state. However, the set of applicable delete records must never change. + * + * @param deleteFile a new delete file + * @param batchOrdinal The batch ordinal to associate with this data file + * @return this for method chaining + */ + default StreamingUpdate addFile(DeleteFile deleteFile, int batchOrdinal) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement addFile"); + } +} diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index 5ab1def51ca0..7683f1d59d84 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -233,6 +233,17 @@ default AppendFiles newFastAppend() { */ RewriteFiles newRewrite(); + /** + * Create a new {@link StreamingUpdate streaming update API} to append sequential upserts to the + * table. + * + * @return a new {@link StreamingUpdate} + */ + default StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement newStreamingUpdate()"); + } + /** * Create a new {@link RewriteManifests rewrite manifests API} to replace manifests for this table * and commit. diff --git a/api/src/main/java/org/apache/iceberg/Transaction.java b/api/src/main/java/org/apache/iceberg/Transaction.java index aeec1f589d06..85d0267bd379 100644 --- a/api/src/main/java/org/apache/iceberg/Transaction.java +++ b/api/src/main/java/org/apache/iceberg/Transaction.java @@ -95,6 +95,17 @@ default AppendFiles newFastAppend() { */ RewriteFiles newRewrite(); + /** + * Create a new {@link StreamingUpdate streaming update API} to append sequential upserts to the + * table. + * + * @return a new {@link StreamingUpdate} + */ + default StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement newStreamingUpdate()"); + } + /** * Create a new {@link RewriteManifests rewrite manifests API} to replace manifests for this * table. diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index cefd765a91b5..fcb528caba96 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -360,6 +360,52 @@ public int hashCode() { } } + // similar to Row but has its own hashCode() and equals() implementations + // it is useful for testing custom collections that rely on wrappers + public static class CustomRow implements StructLike { + public static CustomRow of(Object... values) { + return new CustomRow(values); + } + + private final Object[] values; + + private CustomRow(Object... values) { + this.values = values; + } + + @Override + public int size() { + return values.length; + } + + @Override + public T get(int pos, Class javaClass) { + return javaClass.cast(values[pos]); + } + + @Override + public void set(int pos, T value) { + values[pos] = value; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + CustomRow that = (CustomRow) other; + return Arrays.equals(values, that.values); + } + + @Override + public int hashCode() { + return 17 * Arrays.hashCode(values); + } + } + public static class TestFieldSummary implements ManifestFile.PartitionFieldSummary { private final boolean containsNull; private final Boolean containsNaN; diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java index a6928183f705..2c5f74ad8064 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -122,9 +123,9 @@ public void roundTripSerde() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .build(); @@ -158,9 +159,9 @@ public void roundTripSerdeWithProperties() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .properties(ImmutableMap.of("k1", "v1")) @@ -198,9 +199,9 @@ public void roundTripWithBody() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .properties(ImmutableMap.of("k1", "v1")) diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java index d7337b1b1777..d2cf132ba598 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -70,9 +71,9 @@ public void roundTripSerde() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .build(); diff --git a/build.gradle b/build.gradle index 94996a41a648..3f76cbea02bf 100644 --- a/build.gradle +++ b/build.gradle @@ -577,11 +577,11 @@ project(':iceberg-delta-lake') { exclude group: 'com.google.code.gson', module: 'gson' } - // The newest version of delta-core uses Spark 3.3.*. Since its only for test, we do + // The newest version of delta-core uses Spark 3.5.*. Since its only for test, we do // not need to include older version of delta-core - if (sparkVersions.contains("3.3")) { - integrationImplementation "io.delta:delta-core_${scalaVersion}:${libs.versions.delta.core.get()}" - integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.3_${scalaVersion}") + if (sparkVersions.contains("3.5")) { + integrationImplementation "io.delta:delta-spark_${scalaVersion}:${libs.versions.delta.spark.get()}" + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-3.5_${scalaVersion}") integrationImplementation(libs.hadoop2.minicluster) { exclude group: 'org.apache.avro', module: 'avro' // to make sure netty libs only come from project(':iceberg-arrow') @@ -590,7 +590,7 @@ project(':iceberg-delta-lake') { } integrationImplementation project(path: ':iceberg-hive-metastore') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive33.get()}") { + integrationImplementation("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive35.get()}") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' exclude group: 'org.apache.parquet' @@ -602,9 +602,9 @@ project(':iceberg-delta-lake') { } } - // The newest version of delta-core uses Spark 3.3.*. The integration test should only be built - // if iceberg-spark-3.3 is available - if (sparkVersions.contains("3.3")) { + // The newest version of delta-core uses Spark 3.5.*. The integration test should only be built + // if iceberg-spark-3.5 is available + if (sparkVersions.contains("3.5")) { sourceSets { integration { java.srcDir "$projectDir/src/integration/java" @@ -615,6 +615,7 @@ project(':iceberg-delta-lake') { } task integrationTest(type: Test) { + useJUnitPlatform() testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath jvmArgs += project.property('extraJvmArgs') diff --git a/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java b/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java index 4acbf2a16396..f4b015c16053 100644 --- a/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseReadOnlyTable.java @@ -66,6 +66,11 @@ public RewriteFiles newRewrite() { throw new UnsupportedOperationException("Cannot rewrite in a " + descriptor + " table"); } + @Override + public StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException("Cannot update a " + descriptor + " table"); + } + @Override public RewriteManifests rewriteManifests() { throw new UnsupportedOperationException( diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 87768e34894a..c70dda2bd6d0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -22,8 +22,8 @@ import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -247,7 +247,7 @@ private void performRewrite(List currentManifests) { rewrittenManifests.add(manifest); try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById()) - .select(Arrays.asList("*"))) { + .select(Collections.singletonList("*"))) { reader .liveEntries() .forEach( diff --git a/core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java b/core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java new file mode 100644 index 000000000000..8ef34afb18ec --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/BaseStreamingUpdate.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.util.Comparator; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class BaseStreamingUpdate extends MergingSnapshotProducer + implements StreamingUpdate { + private final List batches = Lists.newArrayList(); + + BaseStreamingUpdate(String tableName, TableOperations ops) { + super(tableName, ops); + } + + @Override + protected BaseStreamingUpdate self() { + return this; + } + + @Override + protected String operation() { + return DataOperations.OVERWRITE; + } + + @Override + public List apply(TableMetadata base, Snapshot snapshot) { + long startingSequenceNumber = base.nextSequenceNumber(); + batches.sort(Comparator.comparingInt(o -> o.ordinal)); + for (Batch batch : batches) { + long dataSequenceNumber = startingSequenceNumber + batch.ordinal + 1; + batch.newDataFiles.forEach(f -> add(f, dataSequenceNumber)); + batch.newDeleteFiles.forEach(f -> add(f, dataSequenceNumber)); + } + return super.apply(base, snapshot); + } + + @Override + public StreamingUpdate addFile(DataFile dataFile, int batchOrdinal) { + return StreamingUpdate.super.addFile(dataFile, batchOrdinal); + } + + @Override + public StreamingUpdate addFile(DeleteFile deleteFile, int batchOrdinal) { + return StreamingUpdate.super.addFile(deleteFile, batchOrdinal); + } + + @Override + public BaseStreamingUpdate toBranch(String branch) { + targetBranch(branch); + return this; + } + + private static class Batch { + private final List newDataFiles = Lists.newArrayList(); + private final List newDeleteFiles = Lists.newArrayList(); + private final int ordinal; + + /** + * Creates a new set of updates to a specific batch + * + * @param ordinal the batch ordinal + */ + Batch(int ordinal) { + this.ordinal = ordinal; + } + + public List getNewDataFiles() { + return newDataFiles; + } + + public List getNewDeleteFiles() { + return newDeleteFiles; + } + + public int getOrdinal() { + return ordinal; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java index d4cf1f4b76f4..ccc018768347 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -190,6 +190,11 @@ public RewriteFiles newRewrite() { return new BaseRewriteFiles(name, ops).reportWith(reporter); } + @Override + public StreamingUpdate newStreamingUpdate() { + return new BaseStreamingUpdate(name, ops).reportWith(reporter); + } + @Override public RewriteManifests rewriteManifests() { return new BaseRewriteManifests(ops).reportWith(reporter); diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 018f70eb16fa..cf263b755002 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -45,6 +45,7 @@ import org.apache.iceberg.metrics.MetricsReporter; 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.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.PropertyUtil; @@ -192,6 +193,16 @@ public RewriteFiles newRewrite() { return rewrite; } + @Override + public StreamingUpdate newStreamingUpdate() { + checkLastOperationCommitted("StreamingUpdate"); + StreamingUpdate streamingUpdate = + new BaseStreamingUpdate(tableName, transactionOps).reportWith(reporter); + streamingUpdate.deleteWith(enqueueDelete); + updates.add(streamingUpdate); + return streamingUpdate; + } + @Override public RewriteManifests rewriteManifests() { checkLastOperationCommitted("RewriteManifests"); @@ -446,16 +457,20 @@ private void commitSimpleTransaction() { } Set committedFiles = committedFiles(ops, newSnapshots); - // delete all of the files that were deleted in the most recent set of operation commits - Tasks.foreach(deletedFiles) - .suppressFailureWhenFinished() - .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) - .run( - path -> { - if (committedFiles == null || !committedFiles.contains(path)) { - ops.io().deleteFile(path); - } - }); + if (committedFiles != null) { + // delete all of the files that were deleted in the most recent set of operation commits + Tasks.foreach(deletedFiles) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) + .run( + path -> { + if (!committedFiles.contains(path)) { + ops.io().deleteFile(path); + } + }); + } else { + LOG.warn("Failed to load metadata for a committed snapshot, skipping clean-up"); + } } catch (RuntimeException e) { LOG.warn("Failed to load committed metadata, skipping clean-up", e); @@ -502,9 +517,11 @@ private void applyUpdates(TableOperations underlyingOps) { } } + // committedFiles returns null whenever the set of committed files + // cannot be determined from the provided snapshots private static Set committedFiles(TableOperations ops, Set snapshotIds) { if (snapshotIds.isEmpty()) { - return null; + return ImmutableSet.of(); } Set committedFiles = Sets.newHashSet(); @@ -696,6 +713,11 @@ public RewriteFiles newRewrite() { return BaseTransaction.this.newRewrite(); } + @Override + public StreamingUpdate newStreamingUpdate() { + return BaseTransaction.this.newStreamingUpdate(); + } + @Override public RewriteManifests rewriteManifests() { return BaseTransaction.this.rewriteManifests(); diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 1dcfa6d3d41d..632958e242dd 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; @@ -83,9 +84,9 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final boolean snapshotIdInheritanceEnabled; // update data - private final List newDataFiles = Lists.newArrayList(); + private final List> newDataFiles = Lists.newArrayList(); private Long newDataFilesDataSequenceNumber; - private final Map> newDeleteFilesBySpec = Maps.newHashMap(); + private final Map>> newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); @@ -158,7 +159,8 @@ protected Expression rowFilter() { } protected List addedDataFiles() { - return ImmutableList.copyOf(newDataFiles); + return ImmutableList.copyOf( + newDataFiles.stream().map(FileHolder::file).collect(Collectors.toList())); } protected void failAnyDelete() { @@ -228,31 +230,41 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - setDataSpec(file); - addedFilesSummary.addedFile(dataSpec(), file); + addDataFile(new FileHolder<>(file)); + } + + /** Add a data file to the new snapshot. */ + protected void add(DataFile file, long dataSequenceNumber) { + Preconditions.checkNotNull(file, "Invalid data file: null"); + addDataFile(new FileHolder<>(file, dataSequenceNumber)); + } + + private void addDataFile(FileHolder dataFile) { + setDataSpec(dataFile.file()); + addedFilesSummary.addedFile(dataSpec(), dataFile.file()); hasNewDataFiles = true; - newDataFiles.add(file); + newDataFiles.add(dataFile); } /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file)); + addDelete(new FileHolder<>(file)); } /** Add a delete file to the new snapshot. */ protected void add(DeleteFile file, long dataSequenceNumber) { Preconditions.checkNotNull(file, "Invalid delete file: null"); - add(new DeleteFileHolder(file, dataSequenceNumber)); + addDelete(new FileHolder<>(file, dataSequenceNumber)); } - private void add(DeleteFileHolder fileHolder) { - int specId = fileHolder.deleteFile().specId(); + private void addDelete(FileHolder fileHolder) { + int specId = fileHolder.file().specId(); PartitionSpec fileSpec = ops.current().spec(specId); - List deleteFiles = + List> deleteFiles = newDeleteFilesBySpec.computeIfAbsent(specId, s -> Lists.newArrayList()); deleteFiles.add(fileHolder); - addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); + addedFilesSummary.addedFile(fileSpec, fileHolder.file()); hasNewDeleteFiles = true; } @@ -960,9 +972,23 @@ private List newDataFilesAsManifests() { RollingManifestWriter writer = newRollingManifestWriter(dataSpec()); try { if (newDataFilesDataSequenceNumber == null) { - newDataFiles.forEach(writer::add); + newDataFiles.forEach( + f -> { + if (f.dataSequenceNumber() == null) { + writer.add(f.file()); + } else { + writer.add(f.file(), f.dataSequenceNumber); + } + }); } else { - newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); + newDataFiles.forEach( + f -> { + if (f.dataSequenceNumber() == null) { + writer.add(f.file(), newDataFilesDataSequenceNumber); + } else { + writer.add(f.file(), f.dataSequenceNumber); + } + }); } } finally { writer.close(); @@ -1006,9 +1032,9 @@ private List newDeleteFilesAsManifests() { deleteFiles.forEach( df -> { if (df.dataSequenceNumber() != null) { - writer.add(df.deleteFile(), df.dataSequenceNumber()); + writer.add(df.file(), df.dataSequenceNumber()); } else { - writer.add(df.deleteFile()); + writer.add(df.file()); } }); } finally { @@ -1132,33 +1158,33 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { } } - private static class DeleteFileHolder { - private final DeleteFile deleteFile; + private static class FileHolder> { + private final T file; private final Long dataSequenceNumber; /** * Wrap a delete file for commit with a given data sequence number * - * @param deleteFile delete file + * @param file content file * @param dataSequenceNumber data sequence number to apply */ - DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { - this.deleteFile = deleteFile; + FileHolder(T file, long dataSequenceNumber) { + this.file = file; this.dataSequenceNumber = dataSequenceNumber; } /** * Wrap a delete file for commit with the latest sequence number * - * @param deleteFile delete file + * @param file the content fle */ - DeleteFileHolder(DeleteFile deleteFile) { - this.deleteFile = deleteFile; + FileHolder(T file) { + this.file = file; this.dataSequenceNumber = null; } - public DeleteFile deleteFile() { - return deleteFile; + public T file() { + return file; } public Long dataSequenceNumber() { diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 778a993c5144..b1fbed43b626 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -331,6 +331,11 @@ public RewriteFiles newRewrite() { throw new UnsupportedOperationException(errorMsg("newRewrite")); } + @Override + public StreamingUpdate newStreamingUpdate() { + throw new UnsupportedOperationException("newStreamingWrite"); + } + @Override public RewriteManifests rewriteManifests() { throw new UnsupportedOperationException(errorMsg("rewriteManifests")); diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java index 80ecf84efa45..fc1f55311175 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java @@ -25,7 +25,10 @@ /** Represents a requirement for a {@link MetadataUpdate} */ public interface UpdateRequirement { - void validate(TableMetadata base); + default void validate(TableMetadata base) { + throw new ValidationException( + "Cannot validate %s against a table", this.getClass().getSimpleName()); + } default void validate(ViewMetadata base) { throw new ValidationException( @@ -62,12 +65,25 @@ public void validate(TableMetadata base) { "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); } } + } + + class AssertViewUUID implements UpdateRequirement { + private final String uuid; + + public AssertViewUUID(String uuid) { + Preconditions.checkArgument(uuid != null, "Invalid required UUID: null"); + this.uuid = uuid; + } + + public String uuid() { + return uuid; + } @Override public void validate(ViewMetadata base) { if (!uuid.equalsIgnoreCase(base.uuid())) { throw new CommitFailedException( - "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); + "Requirement failed: view UUID does not match: expected %s != %s", base.uuid(), uuid); } } } diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java b/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java index 091d4f1fc58c..5c4dc2221290 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirementParser.java @@ -35,6 +35,7 @@ private UpdateRequirementParser() {} // assertion types static final String ASSERT_TABLE_UUID = "assert-table-uuid"; + static final String ASSERT_VIEW_UUID = "assert-view-uuid"; static final String ASSERT_TABLE_DOES_NOT_EXIST = "assert-create"; static final String ASSERT_REF_SNAPSHOT_ID = "assert-ref-snapshot-id"; static final String ASSERT_LAST_ASSIGNED_FIELD_ID = "assert-last-assigned-field-id"; @@ -68,6 +69,7 @@ private UpdateRequirementParser() {} private static final Map, String> TYPES = ImmutableMap., String>builder() .put(UpdateRequirement.AssertTableUUID.class, ASSERT_TABLE_UUID) + .put(UpdateRequirement.AssertViewUUID.class, ASSERT_VIEW_UUID) .put(UpdateRequirement.AssertTableDoesNotExist.class, ASSERT_TABLE_DOES_NOT_EXIST) .put(UpdateRequirement.AssertRefSnapshotID.class, ASSERT_REF_SNAPSHOT_ID) .put(UpdateRequirement.AssertLastAssignedFieldId.class, ASSERT_LAST_ASSIGNED_FIELD_ID) @@ -101,6 +103,9 @@ public static void toJson(UpdateRequirement updateRequirement, JsonGenerator gen case ASSERT_TABLE_UUID: writeAssertTableUUID((UpdateRequirement.AssertTableUUID) updateRequirement, generator); break; + case ASSERT_VIEW_UUID: + writeAssertViewUUID((UpdateRequirement.AssertViewUUID) updateRequirement, generator); + break; case ASSERT_REF_SNAPSHOT_ID: writeAssertRefSnapshotId( (UpdateRequirement.AssertRefSnapshotID) updateRequirement, generator); @@ -159,6 +164,8 @@ public static UpdateRequirement fromJson(JsonNode jsonNode) { return readAssertTableDoesNotExist(jsonNode); case ASSERT_TABLE_UUID: return readAssertTableUUID(jsonNode); + case ASSERT_VIEW_UUID: + return readAssertViewUUID(jsonNode); case ASSERT_REF_SNAPSHOT_ID: return readAssertRefSnapshotId(jsonNode); case ASSERT_LAST_ASSIGNED_FIELD_ID: @@ -182,6 +189,11 @@ private static void writeAssertTableUUID( gen.writeStringField(UUID, requirement.uuid()); } + private static void writeAssertViewUUID( + UpdateRequirement.AssertViewUUID requirement, JsonGenerator gen) throws IOException { + gen.writeStringField(UUID, requirement.uuid()); + } + private static void writeAssertRefSnapshotId( UpdateRequirement.AssertRefSnapshotID requirement, JsonGenerator gen) throws IOException { gen.writeStringField(NAME, requirement.refName()); @@ -231,6 +243,11 @@ private static UpdateRequirement readAssertTableUUID(JsonNode node) { return new UpdateRequirement.AssertTableUUID(uuid); } + private static UpdateRequirement readAssertViewUUID(JsonNode node) { + String uuid = JsonUtil.getString(UUID, node); + return new UpdateRequirement.AssertViewUUID(uuid); + } + private static UpdateRequirement readAssertRefSnapshotId(JsonNode node) { String name = JsonUtil.getString(NAME, node); Long snapshotId = JsonUtil.getLongOrNull(SNAPSHOT_ID, node); diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java index 8a7a761ff2c1..6a5d07d7813d 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java @@ -23,6 +23,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.Sets; +import org.apache.iceberg.view.ViewMetadata; public class UpdateRequirements { @@ -56,6 +57,16 @@ public static List forUpdateTable( return builder.build(); } + public static List forReplaceView( + ViewMetadata base, List metadataUpdates) { + Preconditions.checkArgument(null != base, "Invalid view metadata: null"); + Preconditions.checkArgument(null != metadataUpdates, "Invalid metadata updates: null"); + Builder builder = new Builder(null, false); + builder.require(new UpdateRequirement.AssertViewUUID(base.uuid())); + metadataUpdates.forEach(builder::update); + return builder.build(); + } + private static class Builder { private final TableMetadata base; private final ImmutableList.Builder requirements = ImmutableList.builder(); diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java index 846820a99d9f..d4083420efa6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java @@ -202,6 +202,9 @@ public ErrorResponse parseResponse(int code, String json) { public void accept(ErrorResponse error) { switch (error.code()) { case 400: + if (IllegalArgumentException.class.getSimpleName().equals(error.type())) { + throw new IllegalArgumentException(error.message()); + } throw new BadRequestException("Malformed request: %s", error.message()); case 401: throw new NotAuthorizedException("Not authorized: %s", error.message()); 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 5a55afbfce22..5f660f0f4fe8 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -55,6 +55,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; @@ -91,6 +92,7 @@ import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; import org.apache.iceberg.util.EnvironmentUtil; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.ThreadPools; import org.apache.iceberg.view.BaseView; @@ -122,6 +124,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private final Function, RESTClient> clientBuilder; private final BiFunction, FileIO> ioBuilder; private Cache sessions = null; + private Cache tableSessions = null; private Cache fileIOCloser; private AuthSession catalogAuth = null; private boolean keepTokenRefreshed = true; @@ -196,6 +199,7 @@ public void initialize(String name, Map unresolved) { Map baseHeaders = configHeaders(mergedProps); this.sessions = newSessionCache(mergedProps); + this.tableSessions = newSessionCache(mergedProps); this.keepTokenRefreshed = PropertyUtil.propertyAsBoolean( mergedProps, @@ -241,7 +245,15 @@ private AuthSession session(SessionContext context) { AuthSession session = sessions.get( context.sessionId(), - id -> newSession(context.credentials(), context.properties(), catalogAuth)); + id -> { + Pair> newSession = + newSession(context.credentials(), context.properties(), catalogAuth); + if (null != newSession) { + return newSession.second().get(); + } + + return null; + }); return session != null ? session : catalogAuth; } @@ -702,6 +714,10 @@ public Transaction createTransaction() { @Override public Transaction replaceTransaction() { + if (viewExists(context, ident)) { + throw new AlreadyExistsException("View with same name already exists: %s", ident); + } + LoadTableResponse response = loadInternal(context, ident, snapshotMode); String fullName = fullTableName(ident); @@ -854,7 +870,12 @@ private FileIO tableFileIO(SessionContext context, Map config) { } private AuthSession tableSession(Map tableConf, AuthSession parent) { - AuthSession session = newSession(tableConf, tableConf, parent); + Pair> newSession = newSession(tableConf, tableConf, parent); + if (null == newSession) { + return parent; + } + + AuthSession session = tableSessions.get(newSession.first(), id -> newSession.second().get()); return session != null ? session : parent; } @@ -884,30 +905,46 @@ private static ConfigResponse fetchConfig( return configResponse; } - private AuthSession newSession( + private Pair> newSession( Map credentials, Map properties, AuthSession parent) { if (credentials != null) { // use the bearer token without exchanging if (credentials.containsKey(OAuth2Properties.TOKEN)) { - return AuthSession.fromAccessToken( - client, - tokenRefreshExecutor(), + return Pair.of( credentials.get(OAuth2Properties.TOKEN), - expiresAtMillis(properties), - parent); + () -> + AuthSession.fromAccessToken( + client, + tokenRefreshExecutor(), + credentials.get(OAuth2Properties.TOKEN), + expiresAtMillis(properties), + parent)); } if (credentials.containsKey(OAuth2Properties.CREDENTIAL)) { // fetch a token using the client credentials flow - return AuthSession.fromCredential( - client, tokenRefreshExecutor(), credentials.get(OAuth2Properties.CREDENTIAL), parent); + return Pair.of( + credentials.get(OAuth2Properties.CREDENTIAL), + () -> + AuthSession.fromCredential( + client, + tokenRefreshExecutor(), + credentials.get(OAuth2Properties.CREDENTIAL), + parent)); } for (String tokenType : TOKEN_PREFERENCE_ORDER) { if (credentials.containsKey(tokenType)) { // exchange the token for an access token using the token exchange flow - return AuthSession.fromTokenExchange( - client, tokenRefreshExecutor(), credentials.get(tokenType), tokenType, parent); + return Pair.of( + credentials.get(tokenType), + () -> + AuthSession.fromTokenExchange( + client, + tokenRefreshExecutor(), + credentials.get(tokenType), + tokenType, + parent)); } } } @@ -1170,6 +1207,10 @@ public View createOrReplace() { @Override public View replace() { + if (tableExists(context, identifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); + } + return replace(loadView()); } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java index 48dc075b1305..b4dafaa9031b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java @@ -21,9 +21,8 @@ import java.util.Map; import java.util.Objects; import java.util.function.Supplier; -import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.UpdateRequirements; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.view.ViewMetadata; @@ -62,9 +61,7 @@ public void commit(ViewMetadata base, ViewMetadata metadata) { UpdateTableRequest request = UpdateTableRequest.create( - null, - ImmutableList.of(new UpdateRequirement.AssertTableUUID(base.uuid())), - metadata.changes()); + null, UpdateRequirements.forReplaceView(base, metadata.changes()), metadata.changes()); LoadViewResponse response = client.post( diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionMap.java b/core/src/main/java/org/apache/iceberg/util/PartitionMap.java new file mode 100644 index 000000000000..f16b8aadba0e --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/PartitionMap.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +/** + * A map that uses a pair of spec ID and partition tuple as keys. + * + *

This implementation internally stores provided partition tuples in {@link StructLikeMap} for + * consistent hashing and equals behavior. This ensures that objects of different types that + * represent the same structs are treated as equal keys in the map. + * + *

Note: This map is not designed for concurrent modification by multiple threads. However, it + * supports safe concurrent reads, assuming there are no concurrent writes. + * + *

Note: This map does not support null pairs but supports null as partition tuples. + * + * @param the type of values + */ +public class PartitionMap extends AbstractMap, V> { + + private final Map specs; + private final Map> partitionMaps; + + private PartitionMap(Map specs) { + this.specs = specs; + this.partitionMaps = Maps.newHashMap(); + } + + public static PartitionMap create(Map specs) { + return new PartitionMap<>(specs); + } + + @Override + public int size() { + return partitionMaps.values().stream().mapToInt(Map::size).sum(); + } + + @Override + public boolean isEmpty() { + return partitionMaps.values().stream().allMatch(Map::isEmpty); + } + + @Override + public boolean containsKey(Object key) { + return execute(key, this::containsKey, false /* default value */); + } + + public boolean containsKey(int specId, StructLike struct) { + Map partitionMap = partitionMaps.get(specId); + return partitionMap != null && partitionMap.containsKey(struct); + } + + @Override + public boolean containsValue(Object value) { + return partitionMaps.values().stream().anyMatch(map -> map.containsValue(value)); + } + + @Override + public V get(Object key) { + return execute(key, this::get, null /* default value */); + } + + public V get(int specId, StructLike struct) { + Map partitionMap = partitionMaps.get(specId); + return partitionMap != null ? partitionMap.get(struct) : null; + } + + @Override + public V put(Pair key, V value) { + return put(key.first(), key.second(), value); + } + + public V put(int specId, StructLike struct, V value) { + Map partitionMap = partitionMaps.computeIfAbsent(specId, this::newPartitionMap); + return partitionMap.put(struct, value); + } + + @Override + public void putAll(Map, ? extends V> otherMap) { + otherMap.forEach(this::put); + } + + @Override + public V remove(Object key) { + return execute(key, this::removeKey, null /* default value */); + } + + public V removeKey(int specId, StructLike struct) { + Map partitionMap = partitionMaps.get(specId); + return partitionMap != null ? partitionMap.remove(struct) : null; + } + + @Override + public void clear() { + partitionMaps.clear(); + } + + @Override + public Set> keySet() { + PartitionSet keySet = PartitionSet.create(specs); + + for (Entry> specIdAndPartitionMap : partitionMaps.entrySet()) { + int specId = specIdAndPartitionMap.getKey(); + Map partitionMap = specIdAndPartitionMap.getValue(); + for (StructLike partition : partitionMap.keySet()) { + keySet.add(specId, partition); + } + } + + return Collections.unmodifiableSet(keySet); + } + + @Override + public Collection values() { + List values = Lists.newArrayList(); + + for (Map partitionMap : partitionMaps.values()) { + values.addAll(partitionMap.values()); + } + + return Collections.unmodifiableCollection(values); + } + + @Override + public Set, V>> entrySet() { + Set, V>> entrySet = Sets.newHashSet(); + + for (Entry> specIdAndPartitionMap : partitionMaps.entrySet()) { + int specId = specIdAndPartitionMap.getKey(); + Map partitionMap = specIdAndPartitionMap.getValue(); + for (Entry structAndValue : partitionMap.entrySet()) { + entrySet.add(new PartitionEntry<>(specId, structAndValue)); + } + } + + return Collections.unmodifiableSet(entrySet); + } + + public V computeIfAbsent(int specId, StructLike struct, Supplier valueSupplier) { + Map partitionMap = partitionMaps.computeIfAbsent(specId, this::newPartitionMap); + return partitionMap.computeIfAbsent(struct, key -> valueSupplier.get()); + } + + private Map newPartitionMap(int specId) { + PartitionSpec spec = specs.get(specId); + Preconditions.checkNotNull(spec, "Cannot find spec with ID %s: %s", specId, specs); + return StructLikeMap.create(spec.partitionType()); + } + + @Override + public String toString() { + return partitionMaps.entrySet().stream() + .flatMap(this::toStrings) + .collect(Collectors.joining(", ", "{", "}")); + } + + private Stream toStrings(Entry> entry) { + PartitionSpec spec = specs.get(entry.getKey()); + return entry.getValue().entrySet().stream().map(innerEntry -> toString(spec, innerEntry)); + } + + private String toString(PartitionSpec spec, Entry entry) { + StructLike struct = entry.getKey(); + V value = entry.getValue(); + return spec.partitionToPath(struct) + " -> " + (value == this ? "(this Map)" : value); + } + + private R execute(Object key, BiFunction action, R defaultValue) { + if (key instanceof Pair) { + Object first = ((Pair) key).first(); + Object second = ((Pair) key).second(); + if (first instanceof Integer && (second == null || second instanceof StructLike)) { + return action.apply((Integer) first, (StructLike) second); + } + } else if (key == null) { + throw new NullPointerException(getClass().getName() + " does not support null keys"); + } + + return defaultValue; + } + + private static class PartitionEntry implements Entry, V> { + private final int specId; + private final Entry structAndValue; + + private PartitionEntry(int specId, Entry structAndValue) { + this.specId = specId; + this.structAndValue = structAndValue; + } + + @Override + public Pair getKey() { + return Pair.of(specId, structAndValue.getKey()); + } + + @Override + public V getValue() { + return structAndValue.getValue(); + } + + @Override + public int hashCode() { + return Objects.hash(specId, structAndValue); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + PartitionEntry that = (PartitionEntry) other; + return specId == that.specId && Objects.equals(structAndValue, that.structAndValue); + } + + @Override + public V setValue(V newValue) { + throw new UnsupportedOperationException("Cannot set value"); + } + } +} 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 fdd81ac3d005..eff37fa5a9e8 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.util; +import java.util.AbstractSet; import java.util.Collection; import java.util.Iterator; import java.util.Map; @@ -32,7 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -public class PartitionSet implements Set> { +public class PartitionSet extends AbstractSet> { public static PartitionSet create(Map specsById) { return new PartitionSet(specsById); } @@ -62,7 +63,7 @@ public boolean contains(Object o) { if (o instanceof Pair) { Object first = ((Pair) o).first(); Object second = ((Pair) o).second(); - if (first instanceof Integer && second instanceof StructLike) { + if (first instanceof Integer && (second == null || second instanceof StructLike)) { return contains((Integer) first, (StructLike) second); } } @@ -97,7 +98,7 @@ public boolean remove(Object o) { if (o instanceof Pair) { Object first = ((Pair) o).first(); Object second = ((Pair) o).second(); - if (first instanceof Integer && second instanceof StructLike) { + if (first instanceof Integer && (second == null || second instanceof StructLike)) { return remove((Integer) first, (StructLike) second); } } diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java index 58bd03041331..2bb5fa1c9d40 100644 --- a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java @@ -21,7 +21,6 @@ import java.util.AbstractMap; import java.util.Collection; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.function.Function; import org.apache.iceberg.StructLike; @@ -128,9 +127,9 @@ public Set> entrySet() { private static class StructLikeEntry implements Entry { - private Map.Entry inner; + private final Entry inner; - private StructLikeEntry(Map.Entry inner) { + private StructLikeEntry(Entry inner) { this.inner = inner; } @@ -146,25 +145,19 @@ public R getValue() { @Override public int hashCode() { - int hashCode = getKey().hashCode(); - if (getValue() != null) { - hashCode ^= getValue().hashCode(); - } - return hashCode; + return inner.hashCode(); } @Override - @SuppressWarnings("unchecked") public boolean equals(Object o) { if (this == o) { return true; - } else if (!(o instanceof StructLikeEntry)) { + } else if (o == null || getClass() != o.getClass()) { return false; - } else { - StructLikeEntry that = (StructLikeEntry) o; - return Objects.equals(getKey(), that.getKey()) - && Objects.equals(getValue(), that.getValue()); } + + StructLikeEntry that = (StructLikeEntry) o; + return inner.equals(that.inner); } @Override diff --git a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java index 955bfc2bebf1..6e2d6ff5e864 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseMetastoreViewCatalog.java @@ -23,6 +23,7 @@ import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.Schema; +import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.ViewCatalog; @@ -182,6 +183,10 @@ private View create(ViewOperations ops) { } private View replace(ViewOperations ops) { + if (tableExists(identifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); + } + if (null == ops.current()) { throw new NoSuchViewException("View does not exist: %s", identifier); } @@ -230,4 +235,28 @@ private View replace(ViewOperations ops) { return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } } + + @Override + public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { + return new BaseMetastoreViewCatalogTableBuilder(identifier, schema); + } + + /** The purpose of this class is to add view detection when replacing a table */ + protected class BaseMetastoreViewCatalogTableBuilder extends BaseMetastoreCatalogTableBuilder { + private final TableIdentifier identifier; + + public BaseMetastoreViewCatalogTableBuilder(TableIdentifier identifier, Schema schema) { + super(identifier, schema); + this.identifier = identifier; + } + + @Override + public Transaction replaceTransaction() { + if (viewExists(identifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", identifier); + } + + return super.replaceTransaction(); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java index b356db7ce040..12b63659bcb2 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java @@ -25,6 +25,7 @@ public class ViewProperties { public static final String METADATA_COMPRESSION = "write.metadata.compression-codec"; public static final String METADATA_COMPRESSION_DEFAULT = "gzip"; + public static final String COMMENT = "comment"; private ViewProperties() {} } diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java new file mode 100644 index 000000000000..98a6eafaf8f6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.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; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; + +public class FileGenerationUtil { + + private FileGenerationUtil() {} + + public static DataFile generateDataFile(Table table, StructLike partition) { + 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); + return DataFiles.builder(spec) + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withMetrics(metrics) + .build(); + } + + public static DeleteFile generatePositionDeleteFile(Table table, StructLike partition) { + PartitionSpec spec = table.spec(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + Metrics metrics = generatePositionDeleteMetrics(); + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withMetrics(metrics) + .build(); + } + + public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFile) { + PartitionSpec spec = table.spec(); + StructLike partition = dataFile.partition(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + Metrics metrics = generatePositionDeleteMetrics(dataFile); + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath(path) + .withPartition(partition) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withMetrics(metrics) + .build(); + } + + // mimics the behavior of OutputFileFactory + public static String generateFileName() { + int partitionId = random().nextInt(100_000); + int taskId = random().nextInt(100); + UUID operationId = UUID.randomUUID(); + int fileCount = random().nextInt(1_000); + return String.format("%d-%d-%s-%d.parquet", partitionId, taskId, operationId, fileCount); + } + + public static Metrics generateRandomMetrics(Schema schema) { + long rowCount = generateRowCount(); + Map columnSizes = Maps.newHashMap(); + Map valueCounts = Maps.newHashMap(); + Map nullValueCounts = Maps.newHashMap(); + Map nanValueCounts = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + + for (Types.NestedField column : schema.columns()) { + int fieldId = column.fieldId(); + columnSizes.put(fieldId, generateColumnSize()); + 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)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } + + private static Metrics generatePositionDeleteMetrics(DataFile dataFile) { + long rowCount = generateRowCount(); + Map columnSizes = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + + for (Types.NestedField column : DeleteSchemaUtil.pathPosSchema().columns()) { + int fieldId = column.fieldId(); + columnSizes.put(fieldId, generateColumnSize()); + if (fieldId == MetadataColumns.DELETE_FILE_PATH.fieldId()) { + ByteBuffer bound = Conversions.toByteBuffer(Types.StringType.get(), dataFile.path()); + lowerBounds.put(fieldId, bound); + upperBounds.put(fieldId, bound); + } + } + + return new Metrics( + rowCount, + columnSizes, + null /* no value counts */, + null /* no NULL counts */, + null /* no NaN counts */, + lowerBounds, + upperBounds); + } + + private static Metrics generatePositionDeleteMetrics() { + long rowCount = generateRowCount(); + Map columnSizes = Maps.newHashMap(); + + for (Types.NestedField column : DeleteSchemaUtil.pathPosSchema().columns()) { + int fieldId = column.fieldId(); + columnSizes.put(fieldId, generateColumnSize()); + } + + return new Metrics( + rowCount, + columnSizes, + null /* no value counts */, + null /* no NULL counts */, + null /* no NaN counts */, + null /* no lower bounds */, + null /* no upper bounds */); + } + + private static long generateRowCount() { + return 100_000L + random().nextInt(1000); + } + + private static long generateColumnSize() { + return 1_000_000L + random().nextInt(100_000); + } + + private static long generateValueCount() { + return 100_000L + random().nextInt(100); + } + + private static long generateFileSize() { + return random().nextInt(50_000); + } + + private static Random random() { + return ThreadLocalRandom.current(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/ScanTestBase.java b/core/src/test/java/org/apache/iceberg/ScanTestBase.java index 5e7793939925..48a8ccbaa941 100644 --- a/core/src/test/java/org/apache/iceberg/ScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanTestBase.java @@ -23,7 +23,7 @@ import java.io.File; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.expressions.Expression; @@ -56,7 +56,7 @@ public ScanTestBase(int formatVersion) { @Test public void testTableScanHonorsSelect() { - ScanT scan = newScan().select(Arrays.asList("id")); + ScanT scan = newScan().select(Collections.singletonList("id")); Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get())); @@ -69,20 +69,20 @@ public void testTableScanHonorsSelect() { @Test public void testTableBothProjectAndSelect() { Assertions.assertThatThrownBy( - () -> newScan().select(Arrays.asList("id")).project(SCHEMA.select("data"))) + () -> newScan().select(Collections.singletonList("id")).project(SCHEMA.select("data"))) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot set projection schema when columns are selected"); Assertions.assertThatThrownBy( - () -> newScan().project(SCHEMA.select("data")).select(Arrays.asList("id"))) + () -> newScan().project(SCHEMA.select("data")).select(Collections.singletonList("id"))) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot select columns when projection schema is set"); } @Test public void testTableScanHonorsSelectWithoutCaseSensitivity() { - ScanT scan1 = newScan().caseSensitive(false).select(Arrays.asList("ID")); + ScanT scan1 = newScan().caseSensitive(false).select(Collections.singletonList("ID")); // order of refinements shouldn't matter - ScanT scan2 = newScan().select(Arrays.asList("ID")).caseSensitive(false); + ScanT scan2 = newScan().select(Collections.singletonList("ID")).caseSensitive(false); Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get())); diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index f3b5e7b60c31..4fda388159ec 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Comparators; @@ -270,7 +271,7 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { metrics, new int[] {3}, 1, - Arrays.asList(128L), + Collections.singletonList(128L), ByteBuffer.wrap(new byte[16])); } diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java index 92401b9d9ef5..cd32b9606d55 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java @@ -58,6 +58,25 @@ public void testAssertUUIDToJson() { .isEqualTo(expected); } + @Test + public void testAssertViewUUIDFromJson() { + String requirementType = UpdateRequirementParser.ASSERT_VIEW_UUID; + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + String json = String.format("{\"type\":\"assert-view-uuid\",\"uuid\":\"%s\"}", uuid); + UpdateRequirement expected = new UpdateRequirement.AssertViewUUID(uuid); + assertEquals(requirementType, expected, UpdateRequirementParser.fromJson(json)); + } + + @Test + public void testAssertViewUUIDToJson() { + String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; + String expected = String.format("{\"type\":\"assert-view-uuid\",\"uuid\":\"%s\"}", uuid); + UpdateRequirement actual = new UpdateRequirement.AssertViewUUID(uuid); + Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + .as("AssertViewUUID should convert to the correct JSON value") + .isEqualTo(expected); + } + @Test public void testAssertTableDoesNotExistFromJson() { String requirementType = UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST; @@ -262,6 +281,10 @@ public void assertEquals( (UpdateRequirement.AssertTableUUID) expected, (UpdateRequirement.AssertTableUUID) actual); break; + case UpdateRequirementParser.ASSERT_VIEW_UUID: + compareAssertViewUUID( + (UpdateRequirement.AssertViewUUID) expected, (UpdateRequirement.AssertViewUUID) actual); + break; case UpdateRequirementParser.ASSERT_TABLE_DOES_NOT_EXIST: // Don't cast here as the function explicitly tests that the types are correct, given that // the generated JSON @@ -312,6 +335,15 @@ private static void compareAssertTableUUID( .isEqualTo(expected.uuid()); } + private static void compareAssertViewUUID( + UpdateRequirement.AssertViewUUID expected, UpdateRequirement.AssertViewUUID actual) { + Assertions.assertThat(actual.uuid()) + .as("UUID from JSON should not be null") + .isNotNull() + .as("UUID should parse correctly from JSON") + .isEqualTo(expected.uuid()); + } + // AssertTableDoesNotExist does not have any fields beyond the requirement type, so just check // that the classes // are the same and as expected. diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java index bf3c32628ab0..ed1142441736 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java @@ -26,11 +26,14 @@ import java.util.List; import java.util.UUID; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableViewVersion; +import org.apache.iceberg.view.ViewMetadata; import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -38,12 +41,16 @@ public class TestUpdateRequirements { private final TableMetadata metadata = mock(TableMetadata.class); private final TableMetadata updated = mock(TableMetadata.class); + private final ViewMetadata viewMetadata = mock(ViewMetadata.class); + private final ViewMetadata updatedViewMetadata = mock(ViewMetadata.class); @BeforeEach public void before() { String uuid = UUID.randomUUID().toString(); when(metadata.uuid()).thenReturn(uuid); when(updated.uuid()).thenReturn(uuid); + when(viewMetadata.uuid()).thenReturn(uuid); + when(updatedViewMetadata.uuid()).thenReturn(uuid); } @Test @@ -67,6 +74,14 @@ public void nullCheck() { assertThatThrownBy(() -> UpdateRequirements.forReplaceTable(metadata, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid metadata updates: null"); + + assertThatThrownBy(() -> UpdateRequirements.forReplaceView(null, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid view metadata: null"); + + assertThatThrownBy(() -> UpdateRequirements.forReplaceView(viewMetadata, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid metadata updates: null"); } @Test @@ -87,6 +102,13 @@ public void emptyUpdatesForUpdateAndReplaceTable() { .hasOnlyElementsOfType(UpdateRequirement.AssertTableUUID.class); } + @Test + public void emptyUpdatesForReplaceView() { + assertThat(UpdateRequirements.forReplaceView(viewMetadata, ImmutableList.of())) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertViewUUID.class); + } + @Test public void tableAlreadyExists() { List requirements = UpdateRequirements.forCreateTable(ImmutableList.of()); @@ -129,6 +151,39 @@ public void assignUUIDFailure() { updated.uuid(), metadata.uuid())); } + @Test + public void assignUUIDToView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AssignUUID(viewMetadata.uuid()), + new MetadataUpdate.AssignUUID(UUID.randomUUID().toString()), + new MetadataUpdate.AssignUUID(UUID.randomUUID().toString()))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + + @Test + public void assignUUIDToViewFailure() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, ImmutableList.of(new MetadataUpdate.AssignUUID(viewMetadata.uuid()))); + + when(updatedViewMetadata.uuid()).thenReturn(UUID.randomUUID().toString()); + assertThatThrownBy(() -> requirements.forEach(req -> req.validate(updatedViewMetadata))) + .isInstanceOf(CommitFailedException.class) + .hasMessage( + String.format( + "Requirement failed: view UUID does not match: expected %s != %s", + updatedViewMetadata.uuid(), viewMetadata.uuid())); + } + @Test public void upgradeFormatVersion() { List requirements = @@ -143,6 +198,20 @@ public void upgradeFormatVersion() { assertTableUUID(requirements); } + @Test + public void upgradeFormatVersionForView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(2))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfType(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + @Test public void addSchema() { int lastColumnId = 1; @@ -190,6 +259,25 @@ public void addSchemaFailure() { .hasMessage("Requirement failed: last assigned field id changed: expected id 2 != 3"); } + @Test + public void addSchemaForView() { + int lastColumnId = 1; + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AddSchema(new Schema(), lastColumnId), + new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 1), + new MetadataUpdate.AddSchema(new Schema(), lastColumnId + 2))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + @Test public void setCurrentSchema() { int schemaId = 3; @@ -553,6 +641,33 @@ public void setAndRemoveProperties() { assertTableUUID(requirements); } + @Test + public void setAndRemovePropertiesForView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of(new MetadataUpdate.SetProperties(ImmutableMap.of("test", "test")))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + + requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of(new MetadataUpdate.RemoveProperties(Sets.newHashSet("test")))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + @Test public void setLocation() { List requirements = @@ -567,6 +682,93 @@ public void setLocation() { assertTableUUID(requirements); } + @Test + public void setLocationForView() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, ImmutableList.of(new MetadataUpdate.SetLocation("location"))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + + @Test + public void addViewVersion() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(3) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + + @Test + public void setCurrentViewVersion() { + List requirements = + UpdateRequirements.forReplaceView( + viewMetadata, + ImmutableList.of( + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(3) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(1) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.of("ns")) + .build()), + new MetadataUpdate.SetCurrentViewVersion(2))); + requirements.forEach(req -> req.validate(viewMetadata)); + + assertThat(requirements) + .hasSize(1) + .hasOnlyElementsOfTypes(UpdateRequirement.AssertViewUUID.class); + + assertViewUUID(requirements); + } + private void assertTableUUID(List requirements) { assertThat(requirements) .element(0) @@ -574,4 +776,12 @@ private void assertTableUUID(List requirements) { .extracting(UpdateRequirement.AssertTableUUID::uuid) .isEqualTo(metadata.uuid()); } + + private void assertViewUUID(List requirements) { + assertThat(requirements) + .element(0) + .asInstanceOf(InstanceOfAssertFactories.type(UpdateRequirement.AssertViewUUID.class)) + .extracting(UpdateRequirement.AssertViewUUID::uuid) + .isEqualTo(viewMetadata.uuid()); + } } 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 3de9c7b1d3f7..d0634be16c9f 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -1162,7 +1162,7 @@ public void testTableAuth( // if the table returned a bearer token, there will be no token request if (!tableConfig.containsKey("token")) { // client credentials or token exchange to get a table token - Mockito.verify(adapter, times(2)) + Mockito.verify(adapter, times(1)) .execute( eq(HTTPMethod.POST), eq("v1/oauth/tokens"), diff --git a/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java b/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java new file mode 100644 index 000000000000..268f7eada88d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestPartitionMap.java @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestHelpers.CustomRow; +import org.apache.iceberg.TestHelpers.Row; +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.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionMap { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get())); + private static final PartitionSpec UNPARTITIONED_SPEC = PartitionSpec.unpartitioned(); + private static final PartitionSpec BY_DATA_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").withSpecId(1).build(); + private static final PartitionSpec BY_DATA_CATEGORY_BUCKET_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").bucket("category", 8).withSpecId(3).build(); + private static final Map SPECS = + ImmutableMap.of( + UNPARTITIONED_SPEC.specId(), + UNPARTITIONED_SPEC, + BY_DATA_SPEC.specId(), + BY_DATA_SPEC, + BY_DATA_CATEGORY_BUCKET_SPEC.specId(), + BY_DATA_CATEGORY_BUCKET_SPEC); + + @Test + public void testEmptyMap() { + PartitionMap map = PartitionMap.create(SPECS); + assertThat(map).isEmpty(); + assertThat(map).hasSize(0); + assertThat(map).doesNotContainKey(Pair.of(1, Row.of(1))).doesNotContainValue("value"); + assertThat(map.values()).isEmpty(); + assertThat(map.keySet()).isEmpty(); + assertThat(map.entrySet()).isEmpty(); + } + + @Test + public void testSize() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(UNPARTITIONED_SPEC.specId(), null, "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v2"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v3"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("ccc", 2), "v4"); + assertThat(map).isNotEmpty(); + assertThat(map).hasSize(4); + } + + @Test + public void testDifferentStructLikeImplementations() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "value"); + map.put(UNPARTITIONED_SPEC.specId(), null, "value"); + assertThat(map) + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), CustomRow.of("aaa")), "value") + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa")), "value") + .containsEntry(Pair.of(UNPARTITIONED_SPEC.specId(), null), "value"); + } + + @Test + public void testPutAndGet() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(UNPARTITIONED_SPEC.specId(), null, "v1"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("aaa", 1), "v2"); + assertThat(map.get(UNPARTITIONED_SPEC.specId(), null)).isEqualTo("v1"); + assertThat(map.get(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("aaa", 1))).isEqualTo("v2"); + } + + @Test + public void testRemove() { + PartitionMap map = PartitionMap.create(SPECS); + + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + map.removeKey(BY_DATA_SPEC.specId(), Row.of("aaa")); + + assertThat(map).doesNotContainKey(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa"))); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("aaa"))).isNull(); + assertThat(map).containsKey(Pair.of(BY_DATA_SPEC.specId(), Row.of("bbb"))); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("bbb"))).isEqualTo("v2"); + } + + @Test + public void putAll() { + PartitionMap map = PartitionMap.create(SPECS); + + Map, String> otherMap = Maps.newHashMap(); + otherMap.put(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa")), "v1"); + otherMap.put(Pair.of(BY_DATA_SPEC.specId(), Row.of("bbb")), "v2"); + map.putAll(otherMap); + + assertThat(map) + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), Row.of("aaa")), "v1") + .containsEntry(Pair.of(BY_DATA_SPEC.specId(), Row.of("bbb")), "v2"); + } + + @Test + public void testClear() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(UNPARTITIONED_SPEC.specId(), null, "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v2"); + assertThat(map).hasSize(2); + map.clear(); + assertThat(map).isEmpty(); + } + + @Test + public void testValues() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), 1); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("aaa", 2), 2); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), 3); + assertThat(map.values()).containsAll(ImmutableList.of(1, 2, 3)); + } + + @Test + public void testEntrySet() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("bbb", 2), "v2"); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("ccc"), "v3"); + assertThat(map.entrySet()).hasSize(3); + } + + @Test + public void testKeySet() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("ccc"), "v2"); + assertThat(map.get(BY_DATA_SPEC.specId(), CustomRow.of("aaa"))).isEqualTo("v1"); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("ccc"))).isEqualTo("v2"); + } + + @Test + public void testEqualsAndHashCode() { + PartitionMap map1 = PartitionMap.create(SPECS); + PartitionMap map2 = PartitionMap.create(SPECS); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + + map1.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map1.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "v1"); + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + } + + @Test + public void testToString() { + PartitionMap map = PartitionMap.create(SPECS); + + // empty map + assertThat(map.toString()).isEqualTo("{}"); + + // single entry + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + assertThat(map.toString()).isEqualTo("{data=aaa -> v1}"); + + // multiple entries + map.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + map.put(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("ccc", 2), "v3"); + assertThat(map.toString()) + .contains("data=aaa -> v1") + .contains("data=bbb -> v2") + .contains("data=ccc/category_bucket=2 -> v3"); + } + + @Test + public void testConcurrentReadAccess() throws InterruptedException { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + map.put(UNPARTITIONED_SPEC.specId(), null, "v3"); + map.put(BY_DATA_SPEC.specId(), CustomRow.of("ccc"), "v4"); + + int numThreads = 10; + ExecutorService executorService = Executors.newFixedThreadPool(numThreads); + + // read the map from multiple threads to ensure thread-local wrappers are used + for (int i = 0; i < numThreads; i++) { + executorService.submit( + () -> { + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("aaa"))).isEqualTo("v1"); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("bbb"))).isEqualTo("v2"); + assertThat(map.get(UNPARTITIONED_SPEC.specId(), null)).isEqualTo("v3"); + assertThat(map.get(BY_DATA_SPEC.specId(), Row.of("ccc"))).isEqualTo("v4"); + }); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(1, TimeUnit.MINUTES)).isTrue(); + } + + @Test + public void testNullKey() { + PartitionMap map = PartitionMap.create(SPECS); + assertThatThrownBy(() -> map.put(null, "value")).isInstanceOf(NullPointerException.class); + assertThatThrownBy(() -> map.get(null)).isInstanceOf(NullPointerException.class); + assertThatThrownBy(() -> map.remove(null)).isInstanceOf(NullPointerException.class); + } + + @Test + public void testUnknownSpecId() { + PartitionMap map = PartitionMap.create(SPECS); + assertThatThrownBy(() -> map.put(Integer.MAX_VALUE, null, "value")) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Cannot find spec with ID"); + } + + @Test + public void testUnmodifiableViews() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + + assertThatThrownBy(() -> map.keySet().add(Pair.of(1, null))) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.values().add("other")) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.entrySet().add(null)) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.entrySet().iterator().next().setValue("other")) + .isInstanceOf(UnsupportedOperationException.class); + assertThatThrownBy(() -> map.entrySet().iterator().remove()) + .isInstanceOf(UnsupportedOperationException.class); + } + + @Test + public void testKeyAndEntrySetEquality() { + PartitionMap map1 = PartitionMap.create(SPECS); + PartitionMap map2 = PartitionMap.create(SPECS); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + + map1.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map1.put(BY_DATA_SPEC.specId(), Row.of("bbb"), "v2"); + map1.put(UNPARTITIONED_SPEC.specId(), null, "v3"); + + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("aaa"), "v1"); + map2.put(BY_DATA_SPEC.specId(), CustomRow.of("bbb"), "v2"); + map2.put(UNPARTITIONED_SPEC.specId(), null, "v3"); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + } + + @Test + public void testLookupArbitraryKeyTypes() { + PartitionMap map = PartitionMap.create(SPECS); + map.put(BY_DATA_SPEC.specId(), Row.of("aaa"), "v1"); + map.put(UNPARTITIONED_SPEC.specId(), null, "v2"); + assertThat(map.containsKey("some-string")).isFalse(); + assertThat(map.get("some-string")).isNull(); + assertThat(map.remove("some-string")).isNull(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java b/core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java new file mode 100644 index 000000000000..533e5904237b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestPartitionSet.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers.CustomRow; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestPartitionSet { + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get())); + private static final PartitionSpec UNPARTITIONED_SPEC = PartitionSpec.unpartitioned(); + private static final PartitionSpec BY_DATA_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").withSpecId(1).build(); + private static final PartitionSpec BY_DATA_CATEGORY_BUCKET_SPEC = + PartitionSpec.builderFor(SCHEMA).identity("data").bucket("category", 8).withSpecId(3).build(); + private static final Map SPECS = + ImmutableMap.of( + UNPARTITIONED_SPEC.specId(), + UNPARTITIONED_SPEC, + BY_DATA_SPEC.specId(), + BY_DATA_SPEC, + BY_DATA_CATEGORY_BUCKET_SPEC.specId(), + BY_DATA_CATEGORY_BUCKET_SPEC); + + @Test + public void testGet() { + PartitionSet set = PartitionSet.create(SPECS); + set.add(BY_DATA_SPEC.specId(), Row.of("a")); + set.add(UNPARTITIONED_SPEC.specId(), null); + set.add(UNPARTITIONED_SPEC.specId(), Row.of()); + set.add(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), CustomRow.of("a", 1)); + + assertThat(set).hasSize(4); + assertThat(set.contains(BY_DATA_SPEC.specId(), CustomRow.of("a"))).isTrue(); + assertThat(set.contains(UNPARTITIONED_SPEC.specId(), null)).isTrue(); + assertThat(set.contains(UNPARTITIONED_SPEC.specId(), CustomRow.of())).isTrue(); + assertThat(set.contains(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("a", 1))).isTrue(); + } + + @Test + public void testRemove() { + PartitionSet set = PartitionSet.create(SPECS); + set.add(BY_DATA_SPEC.specId(), Row.of("a")); + set.add(UNPARTITIONED_SPEC.specId(), null); + set.add(UNPARTITIONED_SPEC.specId(), Row.of()); + set.add(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), CustomRow.of("a", 1)); + + assertThat(set).hasSize(4); + assertThat(set.remove(BY_DATA_SPEC.specId(), CustomRow.of("a"))).isTrue(); + assertThat(set.remove(UNPARTITIONED_SPEC.specId(), null)).isTrue(); + assertThat(set.remove(UNPARTITIONED_SPEC.specId(), CustomRow.of())).isTrue(); + assertThat(set.remove(BY_DATA_CATEGORY_BUCKET_SPEC.specId(), Row.of("a", 1))).isTrue(); + assertThat(set).isEmpty(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java b/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java index db4176ce231e..f18c48eaa344 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java +++ b/core/src/test/java/org/apache/iceberg/util/TestStructLikeMap.java @@ -24,6 +24,8 @@ import java.util.Map; import java.util.Set; import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestHelpers.CustomRow; +import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -147,4 +149,40 @@ public void testKeysWithNulls() { assertThat(map.remove(record3)).isEqualTo("aaa"); } + + @Test + public void testEqualsAndHashCode() { + Map map1 = StructLikeMap.create(STRUCT_TYPE); + Map map2 = StructLikeMap.create(STRUCT_TYPE); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + + map1.put(CustomRow.of(1, null), "aaa"); + map1.put(CustomRow.of(2, null), "bbb"); + + map2.put(Row.of(1, null), "aaa"); + map2.put(Row.of(2, null), "bbb"); + + assertThat(map1).isEqualTo(map2); + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + } + + @Test + public void testKeyAndEntrySetEquality() { + Map map1 = StructLikeMap.create(STRUCT_TYPE); + Map map2 = StructLikeMap.create(STRUCT_TYPE); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + + map1.put(CustomRow.of(1, null), "aaa"); + map1.put(CustomRow.of(2, null), "bbb"); + + map2.put(Row.of(1, null), "aaa"); + map2.put(Row.of(2, null), "bbb"); + + assertThat(map1.keySet()).isEqualTo(map2.keySet()); + assertThat(map1.entrySet()).isEqualTo(map2.entrySet()); + } } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java index 613b8a7fddb1..d2372363723b 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -107,7 +107,8 @@ public void readAndWriteValidViewMetadata() throws Exception { .addVersion(version1) .addVersion(version2) .setLocation("s3://bucket/test/location") - .setProperties(ImmutableMap.of("some-key", "some-value")) + .setProperties( + ImmutableMap.of("some-key", "some-value", ViewProperties.COMMENT, "some-comment")) .setCurrentVersionId(2) .upgradeFormatVersion(1) .build(); @@ -218,7 +219,9 @@ public void viewMetadataWithMetadataLocation() throws Exception { .addVersion(version1) .addVersion(version2) .setLocation("s3://bucket/test/location") - .setProperties(ImmutableMap.of("some-key", "some-value")) + .setProperties( + ImmutableMap.of( + "some-key", "some-value", ViewProperties.COMMENT, "some-comment")) .setCurrentVersionId(2) .upgradeFormatVersion(1) .build()) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 8cb77a776220..10d2d7039ce8 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -36,7 +36,6 @@ import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; -import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; @@ -252,7 +251,7 @@ public void createViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .withQuery(trino.dialect(), trino.sql()) .create()) - .isInstanceOf(Exception.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Invalid view version: Cannot add multiple queries for dialect trino"); } @@ -401,16 +400,14 @@ public void replaceTableViaTransactionThatAlreadyExistsAsView() { assertThat(catalog().viewExists(viewIdentifier)).as("View should exist").isTrue(); - // replace transaction requires table existence - // TODO: replace should check whether the table exists as a view assertThatThrownBy( () -> tableCatalog() .buildTable(viewIdentifier, SCHEMA) .replaceTransaction() .commitTransaction()) - .isInstanceOf(NoSuchTableException.class) - .hasMessageStartingWith("Table does not exist: ns.view"); + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View with same name already exists: ns.view"); } @Test @@ -464,8 +461,6 @@ public void replaceViewThatAlreadyExistsAsTable() { assertThat(tableCatalog().tableExists(tableIdentifier)).as("Table should exist").isTrue(); - // replace view requires the view to exist - // TODO: replace should check whether the view exists as a table assertThatThrownBy( () -> catalog() @@ -474,8 +469,8 @@ public void replaceViewThatAlreadyExistsAsTable() { .withDefaultNamespace(tableIdentifier.namespace()) .withQuery("spark", "select * from ns.tbl") .replace()) - .isInstanceOf(NoSuchViewException.class) - .hasMessageStartingWith("View does not exist: ns.table"); + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Table with same name already exists: ns.table"); } @Test diff --git a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json index cf022c5a7098..4b600d6f6b47 100644 --- a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json +++ b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json @@ -2,7 +2,7 @@ "view-uuid": "fa6506c3-7681-40c8-86dc-e36561f83385", "format-version": 1, "location": "s3://bucket/test/location", - "properties": {"some-key": "some-value"}, + "properties": {"some-key": "some-value", "comment": "some-comment"}, "current-schema-id": 0, "schemas": [ { 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 52966a484e68..cebbea65f57f 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 @@ -18,8 +18,10 @@ */ package org.apache.iceberg.delta; +import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.current_date; import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.date_format; import static org.apache.spark.sql.functions.expr; import io.delta.standalone.DeltaLog; @@ -34,7 +36,6 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.Paths; import java.sql.Timestamp; import java.util.Iterator; @@ -42,7 +43,6 @@ import java.util.Map; import java.util.Random; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.commons.codec.DecoderException; import org.apache.commons.codec.net.URLCodec; import org.apache.iceberg.Snapshot; @@ -62,10 +62,8 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +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 TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String SNAPSHOT_SOURCE_PROP = "snapshot_source"; @@ -74,31 +72,22 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String NAMESPACE = "delta_conversion_test"; private static final String defaultSparkCatalog = "spark_catalog"; private static final String icebergCatalogName = "iceberg_hive"; + private static final Map config = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ); private static Dataset typeTestDataFrame; private static Dataset nestedDataFrame; - static Stream parameters() { - return Stream.of( - Arguments.of( - icebergCatalogName, - SparkCatalog.class.getName(), - ImmutableMap.of( - "type", - "hive", - "default-namespace", - "default", - "parquet-enabled", - "true", - "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync - ))); - } - - @TempDir private Path temp; + @TempDir private File tempA; + @TempDir private File tempB; - public TestSnapshotDeltaLakeTable( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); + public TestSnapshotDeltaLakeTable() { + super(icebergCatalogName, SparkCatalog.class.getName(), config); spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); } @@ -115,7 +104,8 @@ public static void beforeClass() { .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) .withColumn("dateCol", date_add(current_date(), 1)) .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) - .withColumn("stringCol", expr("CAST(timestampCol AS STRING)")) + .withColumn("timestampStrCol", expr("CAST(timestampCol AS STRING)")) + .withColumn("stringCol", date_format(col("timestampCol"), "yyyy/M/d")) .withColumn("booleanCol", expr("longCol > 5")) .withColumn("binaryCol", expr("CAST(longCol AS BINARY)")) .withColumn("byteCol", expr("CAST(longCol AS BYTE)")) @@ -160,11 +150,10 @@ public static void afterClass() { spark.sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", NAMESPACE)); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testBasicSnapshotPartitioned() { String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); - String partitionedLocation = temp.toFile().toURI().toString(); + String partitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); @@ -182,13 +171,12 @@ public void testBasicSnapshotPartitioned() { checkIcebergTableLocation(newTableIdentifier, partitionedLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testBasicSnapshotUnpartitioned() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String unpartitionedLocation = temp.toFile().toURI().toString(); + String unpartitionedLocation = tempA.toURI().toString(); - writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); + writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); @@ -204,12 +192,11 @@ public void testBasicSnapshotUnpartitioned() { checkIcebergTableLocation(newTableIdentifier, unpartitionedLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotWithNewLocation() { String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); - String partitionedLocation = temp.toFile().toURI().toString(); - String newIcebergTableLocation = temp.toFile().toURI().toString(); + String partitionedLocation = tempA.toURI().toString(); + String newIcebergTableLocation = tempB.toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); @@ -228,13 +215,12 @@ public void testSnapshotWithNewLocation() { checkIcebergTableLocation(newTableIdentifier, newIcebergTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotWithAdditionalProperties() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String unpartitionedLocation = temp.toFile().toURI().toString(); + String unpartitionedLocation = tempA.toURI().toString(); - writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); + writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); @@ -267,20 +253,18 @@ public void testSnapshotWithAdditionalProperties() { unpartitionedLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotTableWithExternalDataFiles() { String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); String externalDataFilesIdentifier = destName(defaultSparkCatalog, "external_data_files_table"); - String unpartitionedLocation = temp.toFile().toURI().toString(); - String externalDataFilesTableLocation = temp.toFile().toURI().toString(); + String unpartitionedLocation = tempA.toURI().toString(); + String externalDataFilesTableLocation = tempB.toURI().toString(); - writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation, null); + writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); - writeDeltaTable( - nestedDataFrame, externalDataFilesIdentifier, externalDataFilesTableLocation, null); + writeDeltaTable(nestedDataFrame, externalDataFilesIdentifier, externalDataFilesTableLocation); // Add parquet files to default.external_data_files_table. The newly added parquet files // are not at the same location as the table. addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation); @@ -297,13 +281,19 @@ public void testSnapshotTableWithExternalDataFiles() { checkDataFilePathsIntegrity(newTableIdentifier, externalDataFilesTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotSupportedTypes() { String typeTestIdentifier = destName(defaultSparkCatalog, "type_test_table"); - String typeTestTableLocation = temp.toFile().toURI().toString(); + String typeTestTableLocation = tempA.toURI().toString(); - writeDeltaTable(typeTestDataFrame, typeTestIdentifier, typeTestTableLocation, "stringCol"); + writeDeltaTable( + typeTestDataFrame, + typeTestIdentifier, + typeTestTableLocation, + "stringCol", + "timestampStrCol", + "booleanCol", + "longCol"); String newTableIdentifier = destName(icebergCatalogName, "iceberg_type_test_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( @@ -316,13 +306,12 @@ public void testSnapshotSupportedTypes() { checkIcebergTableProperties(newTableIdentifier, ImmutableMap.of(), typeTestTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotVacuumTable() throws IOException { String vacuumTestIdentifier = destName(defaultSparkCatalog, "vacuum_test_table"); - String vacuumTestTableLocation = temp.toFile().toURI().toString(); + String vacuumTestTableLocation = tempA.toURI().toString(); - writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation, null); + writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation); Random random = new Random(); for (int i = 0; i < 13; i++) { spark.sql( @@ -352,11 +341,10 @@ public void testSnapshotVacuumTable() throws IOException { checkIcebergTableLocation(newTableIdentifier, vacuumTestTableLocation); } - @ParameterizedTest(name = "Catalog Name {0} - Options {2}") - @MethodSource("parameters") + @Test public void testSnapshotLogCleanTable() throws IOException { String logCleanTestIdentifier = destName(defaultSparkCatalog, "log_clean_test_table"); - String logCleanTestTableLocation = temp.toFile().toURI().toString(); + String logCleanTestTableLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, logCleanTestIdentifier, logCleanTestTableLocation, "id"); Random random = new Random(); @@ -549,14 +537,14 @@ private static String getFullFilePath(String path, String tableRoot) { } private void writeDeltaTable( - Dataset df, String identifier, String path, String partitionColumn) { + Dataset df, String identifier, String path, String... partitionColumns) { spark.sql(String.format("DROP TABLE IF EXISTS %s", identifier)); - if (partitionColumn != null) { + if (partitionColumns.length > 0) { df.write() .format("delta") .mode(SaveMode.Append) .option("path", path) - .partitionBy(partitionColumn) + .partitionBy(partitionColumns) .saveAsTable(identifier); } else { df.write().format("delta").mode(SaveMode.Append).option("path", path).saveAsTable(identifier); diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java index eb2414ac8276..afe43f6bb165 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/BaseSnapshotDeltaLakeTableAction.java @@ -386,18 +386,18 @@ private DataFile buildDataFileFromAction(Action action, Table table) { nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; Metrics metrics = getMetricsForFile(file, format, metricsConfig, nameMapping); - String partition = + List partitionValueList = spec.fields().stream() .map(PartitionField::name) - .map(name -> String.format("%s=%s", name, partitionValues.get(name))) - .collect(Collectors.joining("/")); + .map(partitionValues::get) + .collect(Collectors.toList()); return DataFiles.builder(spec) .withPath(fullFilePath) .withFormat(format) .withFileSizeInBytes(fileSize) .withMetrics(metrics) - .withPartitionPath(partition) + .withPartitionValues(partitionValueList) .build(); } diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index 8b7b3d30aa26..5509a2ea3223 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -19,7 +19,7 @@ # SCALA_VERSION=2.12 -FLINK_VERSIONS=1.15,1.16,1.17 +FLINK_VERSIONS=1.16,1.17,1.18 SPARK_VERSIONS=3.2,3.3,3.4,3.5 HIVE_VERSIONS=2,3 diff --git a/docs/configuration.md b/docs/configuration.md index ead5c763ab2b..7931394f3c0b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -118,7 +118,7 @@ The value of these properties are not persisted as a part of the table metadata. | Property | Default | Description | | -------------- | -------- | ------------------------------------------------------------- | -| format-version | 1 | Table's format version (can be 1 or 2) as defined in the [Spec](../../../spec/#format-versioning). | +| format-version | 2 | Table's format version (can be 1 or 2) as defined in the [Spec](../../../spec/#format-versioning). Defaults to 2 since version 1.4.0. | ### Compatibility flags diff --git a/docs/flink-queries.md b/docs/flink-queries.md index 4cef5468cd1a..cf68fa367c21 100644 --- a/docs/flink-queries.md +++ b/docs/flink-queries.md @@ -277,6 +277,75 @@ DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks() "Iceberg Source as Avro GenericRecord", new GenericRecordAvroTypeInfo(avroSchema)); ``` +### Emitting watermarks +Emitting watermarks from the source itself could be beneficial for several purposes, like harnessing the +[Flink Watermark Alignment](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment), +or prevent triggering [windows](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/operators/windows/) +too early when reading multiple data files concurrently. + +Enable watermark generation for an `IcebergSource` by setting the `watermarkColumn`. +The supported column types are `timestamp`, `timestamptz` and `long`. +Iceberg `timestamp` or `timestamptz` inherently contains the time precision. So there is no need +to specify the time unit. But `long` type column doesn't contain time unit information. Use +`watermarkTimeUnit` to configure the conversion for long columns. + +The watermarks are generated based on column metrics stored for data files and emitted once per split. +If multiple smaller files with different time ranges are combined into a single split, it can increase +the out-of-orderliness and extra data buffering in the Flink state. The main purpose of watermark alignment +is to reduce out-of-orderliness and excess data buffering in the Flink state. Hence it is recommended to +set `read.split.open-file-cost` to a very large value to prevent combining multiple smaller files into a +single split. The negative impact (of not combining small files into a single split) is on read throughput, +especially if there are many small files. In typical stateful processing jobs, source read throughput is not +the bottleneck. Hence this is probably a reasonable tradeoff. + +This feature requires column-level min-max stats. Make sure stats are generated for the watermark column +during write phase. By default, the column metrics are collected for the first 100 columns of the table. +If watermark column doesn't have stats enabled by default, use +[write properties](configuration.md#write-properties) starting with `write.metadata.metrics` when needed. + +The following example could be useful if watermarks are used for windowing. The source reads Iceberg data files +in order, using a timestamp column and emits watermarks: +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); + +DataStream stream = + env.fromSource( + IcebergSource.forRowData() + .tableLoader(tableLoader) + // Watermark using timestamp column + .watermarkColumn("timestamp_column") + .build(), + // Watermarks are generated by the source, no need to generate it manually + WatermarkStrategy.noWatermarks() + // Extract event timestamp from records + .withTimestampAssigner((record, eventTime) -> record.getTimestamp(pos, precision).getMillisecond()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); +``` + +Example for reading Iceberg table using a long event column for watermark alignment: +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); + +DataStream stream = + env.fromSource( + IcebergSource source = IcebergSource.forRowData() + .tableLoader(tableLoader) + // Disable combining multiple files to a single split + .set(FlinkReadOptions.SPLIT_FILE_OPEN_COST, String.valueOf(TableProperties.SPLIT_SIZE_DEFAULT)) + // Watermark using long column + .watermarkColumn("long_column") + .watermarkTimeUnit(TimeUnit.MILLI_SCALE) + .build(), + // Watermarks are generated by the source, no need to generate it manually + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment(watermarkGroup, maxAllowedWatermarkDrift), + SOURCE_NAME, + TypeInformation.of(RowData.class)); +``` + ## Options ### Read options diff --git a/docs/spark-queries.md b/docs/spark-queries.md index 54b1ee6bd120..38d989f003f6 100644 --- a/docs/spark-queries.md +++ b/docs/spark-queries.md @@ -120,6 +120,17 @@ SELECT * FROM prod.db.table TIMESTAMP AS OF 499162860; SELECT * FROM prod.db.table FOR SYSTEM_TIME AS OF 499162860; ``` +The branch or tag may also be specified using a similar syntax to metadata tables, with `branch_` or `tag_`: + +```sql +SELECT * FROM prod.db.table.`branch_audit-branch`; +SELECT * FROM prod.db.table.`tag_historical-snapshot`; +``` + +(Identifiers with "-" are not valid, and so must be escaped using back quotes.) + +Note that the identifier with branch or tag may not be used in combination with `VERSION AS OF`. + #### DataFrame To select a specific table snapshot or the snapshot at some time in the DataFrame API, Iceberg supports four Spark read options: diff --git a/flink/build.gradle b/flink/build.gradle index 33c2be904546..a33fc84e5727 100644 --- a/flink/build.gradle +++ b/flink/build.gradle @@ -19,10 +19,6 @@ def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") -if (flinkVersions.contains("1.15")) { - apply from: file("$projectDir/v1.15/build.gradle") -} - if (flinkVersions.contains("1.16")) { apply from: file("$projectDir/v1.16/build.gradle") } @@ -30,3 +26,7 @@ if (flinkVersions.contains("1.16")) { 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") +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java deleted file mode 100644 index 072332383ada..000000000000 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.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.api.common.typeutils.TypeSerializer; -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.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -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 DataStatisticsCoordinatorProvider> provider; - private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; - - @Before - public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); - receivingTasks = EventReceivingTasks.createForRunningTasks(); - } - - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); - - RecreateOnResetOperatorCoordinator coordinator = - (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(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(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); - - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); - } - - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator> coordinator) - throws InterruptedException, ExecutionException { - CompletableFuture future = new CompletableFuture<>(); - coordinator.checkpointCoordinator(checkpointId, future); - return future.get(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 28a05201c02f..9d7cf179ab1c 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; /** * DataStatistics defines the interface to collect data distribution information. @@ -29,7 +29,7 @@ * (sketching) can be used. */ @Internal -interface DataStatistics { +interface DataStatistics, S> { /** * Check if data statistics contains any statistics information. @@ -38,12 +38,8 @@ interface DataStatistics { */ boolean isEmpty(); - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); /** * Merge current statistics with other statistics. diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index fcfd79884265..c8ac79c61bf6 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -172,6 +172,7 @@ private void handleDataStatisticRequest(int subtask, DataStatisticsEvent e } } + @SuppressWarnings("FutureReturnValueIgnored") private void sendDataStatisticsToSubtasks( long checkpointId, DataStatistics globalDataStatistics) { callInCoordinatorThread( @@ -339,7 +340,7 @@ private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { Preconditions.checkState( - gateways[subtaskIndex].size() > 0, + !gateways[subtaskIndex].isEmpty(), "Coordinator of %s subtask %d is not ready yet to receive events", operatorName, subtaskIndex); diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index d00d5d2e5aa9..5157a37cf2cd 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -22,7 +22,6 @@ 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.api.java.functions.KeySelector; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; @@ -32,6 +31,12 @@ 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; @@ -45,11 +50,12 @@ class DataStatisticsOperator, S> extends AbstractStreamOperator> implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; private final TypeSerializer> statisticsSerializer; private transient volatile DataStatistics localStatistics; @@ -58,11 +64,13 @@ class DataStatisticsOperator, S> DataStatisticsOperator( String operatorName, - KeySelector keySelector, + Schema schema, + SortOrder sortOrder, OperatorEventGateway operatorEventGateway, TypeSerializer> statisticsSerializer) { this.operatorName = operatorName; - this.keySelector = keySelector; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; this.statisticsSerializer = statisticsSerializer; } @@ -126,10 +134,11 @@ public void handleOperatorEvent(OperatorEvent event) { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord streamRecord) { RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java index 2737b1346f0f..8716cb872d0e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java @@ -76,7 +76,6 @@ static , S> byte[] serializeAggregatedStatistics( return bytes.toByteArray(); } - @SuppressWarnings("unchecked") static , S> AggregatedStatistics deserializeAggregatedStatistics( byte[] bytes, TypeSerializer> statisticsSerializer) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 246b56526fd6..0ffffd9cf49f 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -20,20 +20,20 @@ import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ @Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics> { + private final Map statistics; MapDataStatistics() { this.statistics = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { + MapDataStatistics(Map statistics) { this.statistics = statistics; } @@ -43,9 +43,14 @@ public boolean isEmpty() { } @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } } @Override @@ -54,7 +59,7 @@ public void merge(MapDataStatistics otherStatistics) { } @Override - public Map statistics() { + public Map statistics() { return statistics; } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java index 6d07637b29b3..b6cccd0566fc 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java @@ -29,22 +29,22 @@ 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.table.data.RowData; import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @Internal class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; + extends TypeSerializer>> { + private final MapSerializer mapSerializer; - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); } - MapDataStatisticsSerializer(MapSerializer mapSerializer) { + MapDataStatisticsSerializer(MapSerializer mapSerializer) { this.mapSerializer = mapSerializer; } @@ -55,28 +55,28 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); return (duplicateMapSerializer == mapSerializer) ? this : new MapDataStatisticsSerializer(duplicateMapSerializer); } @Override - public DataStatistics> createInstance() { + public MapDataStatistics createInstance() { return new MapDataStatistics(); } @Override - public DataStatistics> copy(DataStatistics obj) { + public MapDataStatistics copy(DataStatistics> obj) { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); // no need to copy value since it is just a Long newMap.put(newKey, entry.getValue()); } @@ -85,8 +85,9 @@ public DataStatistics> copy(DataStatistics } @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { // not much benefit to reuse return copy(from); } @@ -97,7 +98,9 @@ public int getLength() { } @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics mapStatistics = (MapDataStatistics) obj; @@ -105,14 +108,15 @@ public void serialize(DataStatistics obj, DataOutputView target) throws IOExcept } @Override - public DataStatistics> deserialize(DataInputView source) + public DataStatistics> deserialize(DataInputView source) throws IOException { return new MapDataStatistics(mapSerializer.deserialize(source)); } @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { // not much benefit to reuse return deserialize(source); } @@ -138,14 +142,14 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot>> + public TypeSerializerSnapshot>> snapshotConfiguration() { return new MapDataStatisticsSerializerSnapshot(this); } public static class MapDataStatisticsSerializerSnapshot extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { + DataStatistics>, MapDataStatisticsSerializer> { private static final int CURRENT_VERSION = 1; // constructors need to public. Otherwise, Flink state restore would complain @@ -175,8 +179,8 @@ protected TypeSerializer[] getNestedSerializers( protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; return new MapDataStatisticsSerializer(mapSerializer); } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion == 1) { + readV1(in); + } else { + throw new IllegalArgumentException("Unknown read version: " + readVersion); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index dd7fcafe5336..890cc361b246 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -21,41 +21,43 @@ 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.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.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() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); + mapDataStatistics2.add(keyA); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 48e4e4d8f966..4c64ce522201 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -21,32 +21,33 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.types.Types; import org.junit.Before; import org.junit.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> aggregatedStatisticsTracker; + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + @Before public void before() throws Exception { aggregatedStatisticsTracker = @@ -56,8 +57,8 @@ public void before() throws Exception { @Test public void receiveNewerDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -67,8 +68,8 @@ public void receiveNewerDataStatisticEvent() { assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -82,10 +83,10 @@ public void receiveNewerDataStatisticEvent() { @Test public void receiveOlderDataStatisticEventTest() { MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint3Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -94,8 +95,8 @@ public void receiveOlderDataStatisticEventTest() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return @@ -110,10 +111,10 @@ public void receiveOlderDataStatisticEventTest() { @Test public void receiveCompletedDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -122,14 +123,14 @@ public void receiveCompletedDataStatisticEvent() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = + AggregatedStatistics> completedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion( 1, checkpoint1Subtask1DataStatisticEvent); @@ -137,20 +138,20 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics globalDataStatistics = (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + assertThat((long) globalDataStatistics.statistics().get(keyA)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) .isEqualTo(completedStatistics.checkpointId() + 1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -160,8 +161,8 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint2Subtask1DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 2 diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 9ec2606e101d..3df714059c37 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -24,19 +24,15 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; 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.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; 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; @@ -44,20 +40,21 @@ 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 TypeSerializer>> - statisticsSerializer; + + 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> + private DataStatisticsCoordinator> dataStatisticsCoordinator; @Before public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - dataStatisticsCoordinator = new DataStatisticsCoordinator<>( OPERATOR_NAME, @@ -93,59 +90,66 @@ public void testThrowExceptionWhenNotStarted() { @Test public void testDataStatisticsEventHandling() throws Exception { tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + SortKey key = sortKey.copy(); MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify global data statistics is the aggregation of all subtasks data statistics + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); MapDataStatistics globalDataStatistics = (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); assertThat(globalDataStatistics.statistics()) .containsExactlyInAnyOrderEntriesOf( ImmutableMap.of( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + keyA, 2L, + keyB, 3L, + keyC, 5L)); } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator> dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -154,7 +158,7 @@ static void setAllTasksReady( } static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + DataStatisticsCoordinator> coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index cb9d3f48ffeb..5e0a752be506 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -23,18 +23,14 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; 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.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.types.Types; import org.junit.Before; import org.junit.Test; @@ -42,16 +38,18 @@ public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_SUBTASKS = 1; - private DataStatisticsCoordinatorProvider> provider; + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; @Before public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); provider = new DataStatisticsCoordinatorProvider<>( "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); @@ -61,84 +59,82 @@ public void before() { @Test @SuppressWarnings("unchecked") public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); + 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"); - RecreateOnResetOperatorCoordinator coordinator = + try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + // 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(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } } private byte[] waitForCheckpoint( long checkpointId, - DataStatisticsCoordinator> coordinator) + DataStatisticsCoordinator> coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 880cb3d5516f..0e99a2d74ccb 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -27,7 +27,6 @@ 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.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -50,33 +49,37 @@ 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.binary.BinaryRowData; 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 RowType rowType = RowType.of(new VarCharType()); + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; private Environment getTestingEnvironment() { return new StreamMockEnvironment( @@ -99,20 +102,10 @@ public void before() throws Exception { new MockOutput<>(Lists.newArrayList())); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator> createOperator() { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } @After @@ -123,20 +116,26 @@ public void clean() throws Exception { @Test public void testProcessElement() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); + Map statsMap = mapDataStatistics.statistics(); assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + testHarness.endInput(); } } @@ -144,11 +143,14 @@ public void testProcessElement() throws Exception { @Test public void testOperatorOutput() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); List recordsOutput = testHarness.extractOutputValues().stream() @@ -157,7 +159,10 @@ public void testOperatorOutput() throws Exception { .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); } } @@ -165,36 +170,61 @@ public void testOperatorOutput() throws Exception { public void testRestoreState() throws Exception { OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + .containsExactlyInAnyOrderEntriesOf(expectedMap); snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = + DataStatisticsOperator> restoredOperator = createOperator(); try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); } } @@ -209,18 +239,16 @@ private StateInitializationContext getStateContext() throws Exception { } private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> createHarness( - final DataStatisticsOperator> + final DataStatisticsOperator> dataStatisticsOperator) throws Exception { OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); harness.open(); return harness; } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 0bb2eb7766e9..aa4b9cd79e55 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -40,6 +40,7 @@ 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.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -92,7 +93,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 28a05201c02f..9d7cf179ab1c 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; /** * DataStatistics defines the interface to collect data distribution information. @@ -29,7 +29,7 @@ * (sketching) can be used. */ @Internal -interface DataStatistics { +interface DataStatistics, S> { /** * Check if data statistics contains any statistics information. @@ -38,12 +38,8 @@ interface DataStatistics { */ boolean isEmpty(); - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); /** * Merge current statistics with other statistics. diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index d00d5d2e5aa9..5157a37cf2cd 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -22,7 +22,6 @@ 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.api.java.functions.KeySelector; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; @@ -32,6 +31,12 @@ 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; @@ -45,11 +50,12 @@ class DataStatisticsOperator, S> extends AbstractStreamOperator> implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; private final TypeSerializer> statisticsSerializer; private transient volatile DataStatistics localStatistics; @@ -58,11 +64,13 @@ class DataStatisticsOperator, S> DataStatisticsOperator( String operatorName, - KeySelector keySelector, + Schema schema, + SortOrder sortOrder, OperatorEventGateway operatorEventGateway, TypeSerializer> statisticsSerializer) { this.operatorName = operatorName; - this.keySelector = keySelector; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; this.statisticsSerializer = statisticsSerializer; } @@ -126,10 +134,11 @@ public void handleOperatorEvent(OperatorEvent event) { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord streamRecord) { RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); } 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 index 2737b1346f0f..8716cb872d0e 100644 --- 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 @@ -76,7 +76,6 @@ static , S> byte[] serializeAggregatedStatistics( return bytes.toByteArray(); } - @SuppressWarnings("unchecked") static , S> AggregatedStatistics deserializeAggregatedStatistics( byte[] bytes, TypeSerializer> statisticsSerializer) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 246b56526fd6..0ffffd9cf49f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -20,20 +20,20 @@ import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ @Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics> { + private final Map statistics; MapDataStatistics() { this.statistics = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { + MapDataStatistics(Map statistics) { this.statistics = statistics; } @@ -43,9 +43,14 @@ public boolean isEmpty() { } @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } } @Override @@ -54,7 +59,7 @@ public void merge(MapDataStatistics otherStatistics) { } @Override - public Map statistics() { + public Map statistics() { return statistics; } 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 index 6d07637b29b3..b6cccd0566fc 100644 --- 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 @@ -29,22 +29,22 @@ 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.table.data.RowData; import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @Internal class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; + extends TypeSerializer>> { + private final MapSerializer mapSerializer; - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); } - MapDataStatisticsSerializer(MapSerializer mapSerializer) { + MapDataStatisticsSerializer(MapSerializer mapSerializer) { this.mapSerializer = mapSerializer; } @@ -55,28 +55,28 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); return (duplicateMapSerializer == mapSerializer) ? this : new MapDataStatisticsSerializer(duplicateMapSerializer); } @Override - public DataStatistics> createInstance() { + public MapDataStatistics createInstance() { return new MapDataStatistics(); } @Override - public DataStatistics> copy(DataStatistics obj) { + public MapDataStatistics copy(DataStatistics> obj) { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); // no need to copy value since it is just a Long newMap.put(newKey, entry.getValue()); } @@ -85,8 +85,9 @@ public DataStatistics> copy(DataStatistics } @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { // not much benefit to reuse return copy(from); } @@ -97,7 +98,9 @@ public int getLength() { } @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics mapStatistics = (MapDataStatistics) obj; @@ -105,14 +108,15 @@ public void serialize(DataStatistics obj, DataOutputView target) throws IOExcept } @Override - public DataStatistics> deserialize(DataInputView source) + public DataStatistics> deserialize(DataInputView source) throws IOException { return new MapDataStatistics(mapSerializer.deserialize(source)); } @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { // not much benefit to reuse return deserialize(source); } @@ -138,14 +142,14 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot>> + public TypeSerializerSnapshot>> snapshotConfiguration() { return new MapDataStatisticsSerializerSnapshot(this); } public static class MapDataStatisticsSerializerSnapshot extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { + DataStatistics>, MapDataStatisticsSerializer> { private static final int CURRENT_VERSION = 1; // constructors need to public. Otherwise, Flink state restore would complain @@ -175,8 +179,8 @@ protected TypeSerializer[] getNestedSerializers( protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; return new MapDataStatisticsSerializer(mapSerializer); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion == 1) { + readV1(in); + } else { + throw new IllegalArgumentException("Unknown read version: " + readVersion); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 179253cb3a18..a7ce2db61ffb 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -81,16 +81,18 @@ public class IcebergSource implements Source { private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); + // This table loader can be closed, and it is only safe to use this instance for resource + // independent information (e.g. a table name). Copies of this are required to avoid lifecycle + // management conflicts with the user provided table loader. e.g. a copy of this is required for + // split planning, which uses the underlying io, and should be closed after split planning is + // complete. private final TableLoader tableLoader; private final ScanContext scanContext; private final ReaderFunction readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; private final SerializableRecordEmitter emitter; - - // Can't use SerializableTable as enumerator needs a regular table - // that can discover table changes - private transient Table table; + private final String tableName; IcebergSource( TableLoader tableLoader, @@ -100,17 +102,21 @@ public class IcebergSource implements Source splitComparator, Table table, SerializableRecordEmitter emitter) { + Preconditions.checkNotNull(tableLoader, "tableLoader is required."); + Preconditions.checkNotNull(readerFunction, "readerFunction is required."); + Preconditions.checkNotNull(assignerFactory, "assignerFactory is required."); + Preconditions.checkNotNull(table, "table is required."); this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; - this.table = table; this.emitter = emitter; + this.tableName = table.name(); } String name() { - return "IcebergSource-" + lazyTable().name(); + return "IcebergSource-" + tableName; } private String planningThreadName() { @@ -120,38 +126,26 @@ private String planningThreadName() { // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" // from SourceCoordinatorContext implementation. For now, - is used as // the unique thread pool name. - return lazyTable().name() + "-" + UUID.randomUUID(); + return tableName + "-" + UUID.randomUUID(); } private List planSplitsForBatch(String threadName) { ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); - try { + try (TableLoader loader = tableLoader.clone()) { + loader.open(); List splits = - FlinkSplitPlanner.planIcebergSourceSplits(lazyTable(), scanContext, workerPool); + FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", - splits.size(), - lazyTable().name()); + "Discovered {} splits from table {} during job initialization", splits.size(), tableName); return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to close table loader", e); } finally { workerPool.shutdown(); } } - private Table lazyTable() { - if (table == null) { - tableLoader.open(); - try (TableLoader loader = tableLoader) { - this.table = loader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table loader", e); - } - } - - return table; - } - @Override public Boundedness getBoundedness() { return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; @@ -160,7 +154,7 @@ public Boundedness getBoundedness() { @Override public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = - new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); + new IcebergSourceReaderMetrics(readerContext.metricGroup(), tableName); return new IcebergSourceReader<>( emitter, metrics, readerFunction, splitComparator, readerContext); } @@ -197,13 +191,12 @@ private SplitEnumerator createEnumer LOG.info( "Iceberg source restored {} splits from state for table {}", enumState.pendingSplits().size(), - lazyTable().name()); + tableName); assigner = assignerFactory.createAssigner(enumState.pendingSplits()); } - if (scanContext.isStreaming()) { ContinuousSplitPlanner splitPlanner = - new ContinuousSplitPlannerImpl(tableLoader.clone(), scanContext, planningThreadName()); + new ContinuousSplitPlannerImpl(tableLoader, scanContext, planningThreadName()); return new ContinuousIcebergEnumerator( enumContext, assigner, scanContext, splitPlanner, enumState); } else { @@ -537,7 +530,6 @@ public IcebergSource build() { } } - checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading return new IcebergSource<>( tableLoader, @@ -548,11 +540,5 @@ public IcebergSource build() { table, emitter); } - - private void checkRequired() { - Preconditions.checkNotNull(tableLoader, "tableLoader is required."); - Preconditions.checkNotNull(splitAssignerFactory, "assignerFactory is required."); - Preconditions.checkNotNull(readerFunction, "readerFunction is required."); - } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java index f0d8ca8d7057..450b649253a4 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java @@ -56,9 +56,9 @@ public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { */ public ContinuousSplitPlannerImpl( TableLoader tableLoader, ScanContext scanContext, String threadName) { - this.tableLoader = tableLoader; + this.tableLoader = tableLoader.clone(); this.tableLoader.open(); - this.table = tableLoader.loadTable(); + this.table = this.tableLoader.loadTable(); this.scanContext = scanContext; this.isSharedPool = threadName == null; this.workerPool = diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index dd7fcafe5336..890cc361b246 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -21,41 +21,43 @@ 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.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.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() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); + mapDataStatistics2.add(keyA); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 48e4e4d8f966..4c64ce522201 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -21,32 +21,33 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.types.Types; import org.junit.Before; import org.junit.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> aggregatedStatisticsTracker; + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + @Before public void before() throws Exception { aggregatedStatisticsTracker = @@ -56,8 +57,8 @@ public void before() throws Exception { @Test public void receiveNewerDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -67,8 +68,8 @@ public void receiveNewerDataStatisticEvent() { assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -82,10 +83,10 @@ public void receiveNewerDataStatisticEvent() { @Test public void receiveOlderDataStatisticEventTest() { MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint3Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -94,8 +95,8 @@ public void receiveOlderDataStatisticEventTest() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return @@ -110,10 +111,10 @@ public void receiveOlderDataStatisticEventTest() { @Test public void receiveCompletedDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -122,14 +123,14 @@ public void receiveCompletedDataStatisticEvent() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = + AggregatedStatistics> completedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion( 1, checkpoint1Subtask1DataStatisticEvent); @@ -137,20 +138,20 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics globalDataStatistics = (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + assertThat((long) globalDataStatistics.statistics().get(keyA)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) .isEqualTo(completedStatistics.checkpointId() + 1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -160,8 +161,8 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint2Subtask1DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 2 diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 9ec2606e101d..3df714059c37 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -24,19 +24,15 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; 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.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; 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; @@ -44,20 +40,21 @@ 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 TypeSerializer>> - statisticsSerializer; + + 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> + private DataStatisticsCoordinator> dataStatisticsCoordinator; @Before public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - dataStatisticsCoordinator = new DataStatisticsCoordinator<>( OPERATOR_NAME, @@ -93,59 +90,66 @@ public void testThrowExceptionWhenNotStarted() { @Test public void testDataStatisticsEventHandling() throws Exception { tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + SortKey key = sortKey.copy(); MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify global data statistics is the aggregation of all subtasks data statistics + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); MapDataStatistics globalDataStatistics = (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); assertThat(globalDataStatistics.statistics()) .containsExactlyInAnyOrderEntriesOf( ImmutableMap.of( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + keyA, 2L, + keyB, 3L, + keyC, 5L)); } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator> dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -154,7 +158,7 @@ static void setAllTasksReady( } static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + DataStatisticsCoordinator> coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index cb9d3f48ffeb..5e0a752be506 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -23,18 +23,14 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; 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.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.types.Types; import org.junit.Before; import org.junit.Test; @@ -42,16 +38,18 @@ public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_SUBTASKS = 1; - private DataStatisticsCoordinatorProvider> provider; + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - private TypeSerializer>> - statisticsSerializer; @Before public void before() { - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); provider = new DataStatisticsCoordinatorProvider<>( "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); @@ -61,84 +59,82 @@ public void before() { @Test @SuppressWarnings("unchecked") public void testCheckpointAndReset() throws Exception { - RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - BinaryRowData binaryRowDataD = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("d"))); - BinaryRowData binaryRowDataE = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("e"))); + 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"); - RecreateOnResetOperatorCoordinator coordinator = + try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS)); - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + // 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(binaryRowDataD); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - checkpoint2Subtask0DataStatistic.add(binaryRowDataE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } } private byte[] waitForCheckpoint( long checkpointId, - DataStatisticsCoordinator> coordinator) + DataStatisticsCoordinator> coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 880cb3d5516f..0e99a2d74ccb 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -27,7 +27,6 @@ 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.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -50,33 +49,37 @@ 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.binary.BinaryRowData; 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 RowType rowType = RowType.of(new VarCharType()); + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; private Environment getTestingEnvironment() { return new StreamMockEnvironment( @@ -99,20 +102,10 @@ public void before() throws Exception { new MockOutput<>(Lists.newArrayList())); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator> createOperator() { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } @After @@ -123,20 +116,26 @@ public void clean() throws Exception { @Test public void testProcessElement() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); + Map statsMap = mapDataStatistics.statistics(); assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + testHarness.endInput(); } } @@ -144,11 +143,14 @@ public void testProcessElement() throws Exception { @Test public void testOperatorOutput() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); List recordsOutput = testHarness.extractOutputValues().stream() @@ -157,7 +159,10 @@ public void testOperatorOutput() throws Exception { .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); } } @@ -165,36 +170,61 @@ public void testOperatorOutput() throws Exception { public void testRestoreState() throws Exception { OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + .containsExactlyInAnyOrderEntriesOf(expectedMap); snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = + DataStatisticsOperator> restoredOperator = createOperator(); try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); } } @@ -209,18 +239,16 @@ private StateInitializationContext getStateContext() throws Exception { } private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> createHarness( - final DataStatisticsOperator> + final DataStatisticsOperator> dataStatisticsOperator) throws Exception { OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + 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 new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index aa4b9cd79e55..0bb2eb7766e9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -40,7 +40,6 @@ 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.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -93,10 +92,7 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration( - reporter.addToConfiguration( - // disable classloader check as Avro may cache class in the serializers. - new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) + .setConfiguration(reporter.addToConfiguration(new Configuration())) .withHaLeadershipControl() .build()); diff --git a/flink/v1.15/build.gradle b/flink/v1.18/build.gradle similarity index 91% rename from flink/v1.15/build.gradle rename to flink/v1.18/build.gradle index a77ec1b50ab3..55578d3b117d 100644 --- a/flink/v1.15/build.gradle +++ b/flink/v1.18/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.15' +String flinkMajorVersion = '1.18' 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.flink115.avro + compileOnly libs.flink118.avro // for dropwizard histogram metrics implementation - compileOnly libs.flink115.metrics.dropwizard - compileOnly libs.flink115.streaming.java - compileOnly "${libs.flink115.streaming.java.get().module}:${libs.flink115.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink115.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink115.get()}" - compileOnly libs.flink115.connector.base - compileOnly libs.flink115.connector.files + compileOnly libs.flink118.metrics.dropwizard + compileOnly libs.flink118.streaming.java + compileOnly "${libs.flink118.streaming.java.get().module}:${libs.flink118.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink118.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" + compileOnly libs.flink118.connector.base + compileOnly libs.flink118.connector.files compileOnly libs.hadoop2.hdfs compileOnly libs.hadoop2.common @@ -65,13 +65,13 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.apache.hive', module: 'hive-storage-api' } - testImplementation libs.flink115.connector.test.utils - testImplementation libs.flink115.core - testImplementation libs.flink115.runtime - testImplementation(libs.flink115.test.utilsjunit) { + testImplementation libs.flink118.connector.test.utils + testImplementation libs.flink118.core + testImplementation libs.flink118.runtime + testImplementation(libs.flink118.test.utilsjunit) { exclude group: 'junit' } - testImplementation(libs.flink115.test.utils) { + testImplementation(libs.flink118.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -164,7 +164,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink115.metrics.dropwizard + implementation libs.flink118.metrics.dropwizard // for integration testing with the flink-runtime-jar // all of those dependencies are required because the integration test extends FlinkTestBase @@ -174,13 +174,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.flink115.test.utils) { + integrationImplementation(libs.flink118.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } - integrationImplementation libs.flink115.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink115.get()}" + integrationImplementation libs.flink118.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" integrationImplementation libs.hadoop2.common integrationImplementation libs.hadoop2.hdfs diff --git a/flink/v1.15/flink-runtime/LICENSE b/flink/v1.18/flink-runtime/LICENSE similarity index 100% rename from flink/v1.15/flink-runtime/LICENSE rename to flink/v1.18/flink-runtime/LICENSE diff --git a/flink/v1.15/flink-runtime/NOTICE b/flink/v1.18/flink-runtime/NOTICE similarity index 100% rename from flink/v1.15/flink-runtime/NOTICE rename to flink/v1.18/flink-runtime/NOTICE diff --git a/flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java similarity index 100% rename from flink/v1.15/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java rename to flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java similarity index 89% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index 825816fdf416..f022c8abcb00 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -38,6 +38,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; @@ -60,8 +61,6 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.UpdateProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; @@ -69,6 +68,7 @@ import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.flink.util.FlinkAlterTableUtil; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -91,7 +91,6 @@ * independent of the partition of Flink. */ public class FlinkCatalog extends AbstractCatalog { - private final CatalogLoader catalogLoader; private final Catalog icebergCatalog; private final Namespace baseNamespace; @@ -439,14 +438,35 @@ private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTab if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) && equalsPrimary)) { - throw new UnsupportedOperationException("Altering schema is not supported yet."); + throw new UnsupportedOperationException( + "Altering schema is not supported in the old alterTable API. " + + "To alter schema, use the other alterTable API and provide a list of TableChange's."); } + validateTablePartition(ct1, ct2); + } + + private static void validateTablePartition(CatalogTable ct1, CatalogTable ct2) { if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { throw new UnsupportedOperationException("Altering partition keys is not supported yet."); } } + /** + * This alterTable API only supports altering table properties. + * + *

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

To alter columns, use the other alterTable API and provide a list of TableChange's. + * + * @param tablePath path of the table or view to be modified + * @param newTable the new table definition + * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if set + * to false, throw an exception, if set to true, do nothing. + * @throws CatalogException in case of any runtime exception + * @throws TableNotExistException if the table does not exist + */ @Override public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) throws CatalogException, TableNotExistException { @@ -464,12 +484,6 @@ public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean } CatalogTable table = toCatalogTable(icebergTable); - - // Currently, Flink SQL only support altering table properties. - - // For current Flink Catalog API, support for adding/removing/renaming columns cannot be done by - // comparing - // CatalogTable instances, unless the Flink schema contains Iceberg column IDs. validateTableSchemaAndPartition(table, (CatalogTable) newTable); Map oldProperties = table.getOptions(); @@ -507,7 +521,66 @@ public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean } }); - commitChanges(icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); + FlinkAlterTableUtil.commitChanges( + icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); + } + + @Override + public void alterTable( + ObjectPath tablePath, + CatalogBaseTable newTable, + List tableChanges, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + // Does not support altering partition yet. + validateTablePartition(toCatalogTable(icebergTable), (CatalogTable) newTable); + + String setLocation = null; + String setSnapshotId = null; + String cherrypickSnapshotId = null; + + List propertyChanges = Lists.newArrayList(); + List schemaChanges = Lists.newArrayList(); + for (TableChange change : tableChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption set = (TableChange.SetOption) change; + + if ("location".equalsIgnoreCase(set.getKey())) { + setLocation = set.getValue(); + } else if ("current-snapshot-id".equalsIgnoreCase(set.getKey())) { + setSnapshotId = set.getValue(); + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.getKey())) { + cherrypickSnapshotId = set.getValue(); + } else { + propertyChanges.add(change); + } + } else if (change instanceof TableChange.ResetOption) { + propertyChanges.add(change); + } else { + schemaChanges.add(change); + } + } + + FlinkAlterTableUtil.commitChanges( + icebergTable, + setLocation, + setSnapshotId, + cherrypickSnapshotId, + schemaChanges, + propertyChanges); } private static void validateFlinkTable(CatalogBaseTable table) { @@ -552,52 +625,6 @@ private static List toPartitionKeys(PartitionSpec spec, Schema icebergSc return partitionKeysBuilder.build(); } - private static void commitChanges( - Table table, - String setLocation, - String setSnapshotId, - String pickSnapshotId, - Map setProperties) { - // don't allow setting the snapshot and picking a commit at the same time because order is - // ambiguous and choosing - // one order leads to different results - Preconditions.checkArgument( - setSnapshotId == null || pickSnapshotId == null, - "Cannot set the current snapshot ID and cherry-pick snapshot changes"); - - if (setSnapshotId != null) { - long newSnapshotId = Long.parseLong(setSnapshotId); - table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); - } - - // if updating the table snapshot, perform that update first in case it fails - if (pickSnapshotId != null) { - long newSnapshotId = Long.parseLong(pickSnapshotId); - table.manageSnapshots().cherrypick(newSnapshotId).commit(); - } - - Transaction transaction = table.newTransaction(); - - if (setLocation != null) { - transaction.updateLocation().setLocation(setLocation).commit(); - } - - if (!setProperties.isEmpty()) { - UpdateProperties updateProperties = transaction.updateProperties(); - setProperties.forEach( - (k, v) -> { - if (v == null) { - updateProperties.remove(k); - } else { - updateProperties.set(k, v); - } - }); - updateProperties.commit(); - } - - transaction.commitTransaction(); - } - static CatalogTable toCatalogTable(Table table) { TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); List partitionKeys = toPartitionKeys(table.spec(), table.schema()); diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java similarity index 96% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 25725639c330..a6b53879ad80 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -134,6 +134,16 @@ public static LogicalType convert(Type type) { return TypeUtil.visit(type, new TypeToFlinkType()); } + /** + * Convert a {@link LogicalType Flink type} to a {@link Type}. + * + * @param flinkType a FlinkType + * @return the equivalent Iceberg type + */ + public static Type convert(LogicalType flinkType) { + return flinkType.accept(new FlinkTypeToType()); + } + /** * Convert a {@link RowType} to a {@link TableSchema}. * diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java similarity index 99% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java index 6f8bfef2ef44..408065f06057 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java @@ -49,6 +49,10 @@ class FlinkTypeToType extends FlinkTypeVisitor { private final RowType root; private int nextId; + FlinkTypeToType() { + this.root = null; + } + FlinkTypeToType(RowType root) { this.root = root; // the root struct's fields use the first ids diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/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/AggregatedStatistics.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java diff --git a/flink/v1.15/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 similarity index 87% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 28a05201c02f..9d7cf179ab1c 100644 --- a/flink/v1.15/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 @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; /** * DataStatistics defines the interface to collect data distribution information. @@ -29,7 +29,7 @@ * (sketching) can be used. */ @Internal -interface DataStatistics { +interface DataStatistics, S> { /** * Check if data statistics contains any statistics information. @@ -38,12 +38,8 @@ interface DataStatistics { */ boolean isEmpty(); - /** - * Add data key to data statistics. - * - * @param key generate from data by applying key selector - */ - void add(RowData key); + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); /** * Merge current statistics with other statistics. diff --git a/flink/v1.15/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 similarity index 74% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 26fb3c328481..c8ac79c61bf6 100644 --- a/flink/v1.15/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.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -35,6 +36,8 @@ import org.apache.flink.util.ThrowableCatchingRunnable; import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -54,7 +57,7 @@ class DataStatisticsCoordinator, S> implements Op private final String operatorName; private final ExecutorService coordinatorExecutor; private final OperatorCoordinator.Context operatorCoordinatorContext; - private final OperatorCoordinator.SubtaskGateway[] subtaskGateways; + private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; private final TypeSerializer> statisticsSerializer; private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; @@ -71,8 +74,7 @@ class DataStatisticsCoordinator, S> implements Op "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); this.operatorCoordinatorContext = context; - this.subtaskGateways = - new OperatorCoordinator.SubtaskGateway[operatorCoordinatorContext.currentParallelism()]; + this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); this.statisticsSerializer = statisticsSerializer; this.aggregatedStatisticsTracker = new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); @@ -170,6 +172,7 @@ private void handleDataStatisticRequest(int subtask, DataStatisticsEvent e } } + @SuppressWarnings("FutureReturnValueIgnored") private void sendDataStatisticsToSubtasks( long checkpointId, DataStatistics globalDataStatistics) { callInCoordinatorThread( @@ -178,7 +181,7 @@ private void sendDataStatisticsToSubtasks( DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); int parallelism = parallelism(); for (int i = 0; i < parallelism; ++i) { - subtaskGateways[i].sendEvent(dataStatisticsEvent); + subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); } return null; @@ -189,14 +192,22 @@ private void sendDataStatisticsToSubtasks( } @Override - public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception { + @SuppressWarnings("unchecked") + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { runInCoordinatorThread( () -> { - LOG.debug("Handling event from subtask {} of {}: {}", subtask, operatorName, event); + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); Preconditions.checkArgument(event instanceof DataStatisticsEvent); handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); }, - String.format("handling operator event %s from subtask %d", event.getClass(), subtask)); + String.format( + "handling operator event %s from subtask %d (#%d)", + event.getClass(), subtask, attemptNumber)); } @Override @@ -238,39 +249,49 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData } @Override - public void subtaskFailed(int subtask, @Nullable Throwable reason) { + public void subtaskReset(int subtask, long checkpointId) { runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} of data statistic {}", + "Operator {} subtask {} is reset to checkpoint {}", + operatorName, subtask, - operatorName); + checkpointId); Preconditions.checkState( this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways[subtask] = null; + subtaskGateways.reset(subtask); }, - String.format("handling subtask %d failure", subtask)); + String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override - public void subtaskReset(int subtask, long checkpointId) { - LOG.info( - "Data statistic coordinator {} subtask {} is reset to checkpoint {}", - operatorName, - subtask, - checkpointId); + public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Throwable reason) { + runInCoordinatorThread( + () -> { + LOG.info( + "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + subtask, + attemptNumber, + operatorName); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); + }, + String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override - public void subtaskReady(int subtask, SubtaskGateway gateway) { + public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { Preconditions.checkArgument(subtask == gateway.getSubtask()); + Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); runInCoordinatorThread( () -> { Preconditions.checkState( this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways[subtask] = gateway; + subtaskGateways.registerSubtaskGateway(gateway); }, - String.format("making event gateway to subtask %d available", subtask)); + String.format( + "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); } @VisibleForTesting @@ -278,6 +299,59 @@ AggregatedStatistics completedStatistics() { return completedStatistics; } + private static class SubtaskGateways { + private final String operatorName; + private final Map[] gateways; + + private SubtaskGateways(String operatorName, int parallelism) { + this.operatorName = operatorName; + gateways = new Map[parallelism]; + + for (int i = 0; i < parallelism; ++i) { + gateways[i] = Maps.newHashMap(); + } + } + + private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway gateway) { + int subtaskIndex = gateway.getSubtask(); + int attemptNumber = gateway.getExecution().getAttemptNumber(); + Preconditions.checkState( + !gateways[subtaskIndex].containsKey(attemptNumber), + "Coordinator of %s already has a subtask gateway for %d (#%d)", + operatorName, + subtaskIndex, + attemptNumber); + LOG.debug( + "Coordinator of {} registers gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].put(attemptNumber, gateway); + } + + private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { + LOG.debug( + "Coordinator of {} unregisters gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].remove(attemptNumber); + } + + private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { + Preconditions.checkState( + !gateways[subtaskIndex].isEmpty(), + "Coordinator of %s subtask %d is not ready yet to receive events", + operatorName, + subtaskIndex); + return Iterables.getOnlyElement(gateways[subtaskIndex].values()); + } + + private void reset(int subtaskIndex) { + gateways[subtaskIndex].clear(); + } + } + private static class CoordinatorExecutorThreadFactory implements ThreadFactory, Thread.UncaughtExceptionHandler { diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java similarity index 100% rename from flink/v1.15/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/DataStatisticsEvent.java diff --git a/flink/v1.15/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 similarity index 92% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index d00d5d2e5aa9..5157a37cf2cd 100644 --- a/flink/v1.15/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 @@ -22,7 +22,6 @@ 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.api.java.functions.KeySelector; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; @@ -32,6 +31,12 @@ 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; @@ -45,11 +50,12 @@ class DataStatisticsOperator, S> extends AbstractStreamOperator> implements OneInputStreamOperator>, OperatorEventHandler { + private static final long serialVersionUID = 1L; private final String operatorName; - // keySelector will be used to generate key from data for collecting data statistics - private final KeySelector keySelector; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; private final TypeSerializer> statisticsSerializer; private transient volatile DataStatistics localStatistics; @@ -58,11 +64,13 @@ class DataStatisticsOperator, S> DataStatisticsOperator( String operatorName, - KeySelector keySelector, + Schema schema, + SortOrder sortOrder, OperatorEventGateway operatorEventGateway, TypeSerializer> statisticsSerializer) { this.operatorName = operatorName; - this.keySelector = keySelector; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; this.statisticsSerializer = statisticsSerializer; } @@ -126,10 +134,11 @@ public void handleOperatorEvent(OperatorEvent event) { } @Override - public void processElement(StreamRecord streamRecord) throws Exception { + public void processElement(StreamRecord streamRecord) { RowData record = streamRecord.getValue(); - RowData key = keySelector.getKey(record); - localStatistics.add(key); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java similarity index 100% rename from flink/v1.15/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/DataStatisticsOrRecord.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java similarity index 100% rename from flink/v1.15/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/DataStatisticsOrRecordSerializer.java diff --git a/flink/v1.15/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 similarity index 99% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java index 2737b1346f0f..8716cb872d0e 100644 --- a/flink/v1.15/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 @@ -76,7 +76,6 @@ static , S> byte[] serializeAggregatedStatistics( return bytes.toByteArray(); } - @SuppressWarnings("unchecked") static , S> AggregatedStatistics deserializeAggregatedStatistics( byte[] bytes, TypeSerializer> statisticsSerializer) diff --git a/flink/v1.15/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 similarity index 76% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 246b56526fd6..0ffffd9cf49f 100644 --- a/flink/v1.15/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 @@ -20,20 +20,20 @@ import java.util.Map; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ @Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics> { + private final Map statistics; MapDataStatistics() { this.statistics = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { + MapDataStatistics(Map statistics) { this.statistics = statistics; } @@ -43,9 +43,14 @@ public boolean isEmpty() { } @Override - public void add(RowData key) { - // increase count of occurrence by one in the dataStatistics map - statistics.merge(key, 1L, Long::sum); + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } } @Override @@ -54,7 +59,7 @@ public void merge(MapDataStatistics otherStatistics) { } @Override - public Map statistics() { + public Map statistics() { return statistics; } diff --git a/flink/v1.15/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 similarity index 74% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java index 6d07637b29b3..b6cccd0566fc 100644 --- a/flink/v1.15/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 @@ -29,22 +29,22 @@ 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.table.data.RowData; import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @Internal class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; + extends TypeSerializer>> { + private final MapSerializer mapSerializer; - static TypeSerializer>> fromKeySerializer( - TypeSerializer keySerializer) { + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { return new MapDataStatisticsSerializer( - new MapSerializer<>(keySerializer, LongSerializer.INSTANCE)); + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); } - MapDataStatisticsSerializer(MapSerializer mapSerializer) { + MapDataStatisticsSerializer(MapSerializer mapSerializer) { this.mapSerializer = mapSerializer; } @@ -55,28 +55,28 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); return (duplicateMapSerializer == mapSerializer) ? this : new MapDataStatisticsSerializer(duplicateMapSerializer); } @Override - public DataStatistics> createInstance() { + public MapDataStatistics createInstance() { return new MapDataStatistics(); } @Override - public DataStatistics> copy(DataStatistics obj) { + public MapDataStatistics copy(DataStatistics> obj) { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - RowData newKey = keySerializer.copy(entry.getKey()); + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); // no need to copy value since it is just a Long newMap.put(newKey, entry.getValue()); } @@ -85,8 +85,9 @@ public DataStatistics> copy(DataStatistics } @Override - public DataStatistics> copy( - DataStatistics from, DataStatistics reuse) { + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { // not much benefit to reuse return copy(from); } @@ -97,7 +98,9 @@ public int getLength() { } @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { Preconditions.checkArgument( obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); MapDataStatistics mapStatistics = (MapDataStatistics) obj; @@ -105,14 +108,15 @@ public void serialize(DataStatistics obj, DataOutputView target) throws IOExcept } @Override - public DataStatistics> deserialize(DataInputView source) + public DataStatistics> deserialize(DataInputView source) throws IOException { return new MapDataStatistics(mapSerializer.deserialize(source)); } @Override - public DataStatistics> deserialize( - DataStatistics reuse, DataInputView source) throws IOException { + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { // not much benefit to reuse return deserialize(source); } @@ -138,14 +142,14 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot>> + public TypeSerializerSnapshot>> snapshotConfiguration() { return new MapDataStatisticsSerializerSnapshot(this); } public static class MapDataStatisticsSerializerSnapshot extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { + DataStatistics>, MapDataStatisticsSerializer> { private static final int CURRENT_VERSION = 1; // constructors need to public. Otherwise, Flink state restore would complain @@ -175,8 +179,8 @@ protected TypeSerializer[] getNestedSerializers( protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; return new MapDataStatisticsSerializer(mapSerializer); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + if (readVersion == 1) { + readV1(in); + } else { + throw new IllegalArgumentException("Unknown read version: " + readVersion); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java new file mode 100644 index 000000000000..f0b9bf64fb1a --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.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.util; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; + +public class FlinkAlterTableUtil { + private FlinkAlterTableUtil() {} + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + Map setProperties) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!setProperties.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + setProperties.forEach( + (k, v) -> { + if (v == null) { + updateProperties.remove(k); + } else { + updateProperties.set(k, v); + } + }); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + List schemaChanges, + List propertyChanges) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!schemaChanges.isEmpty()) { + UpdateSchema updateSchema = transaction.updateSchema(); + FlinkAlterTableUtil.applySchemaChanges(updateSchema, schemaChanges); + updateSchema.commit(); + } + + if (!propertyChanges.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + FlinkAlterTableUtil.applyPropertyChanges(updateProperties, propertyChanges); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitManageSnapshots( + Table table, String setSnapshotId, String cherrypickSnapshotId) { + // don't allow setting the snapshot and picking a commit at the same time because order is + // ambiguous and choosing one order leads to different results + Preconditions.checkArgument( + setSnapshotId == null || cherrypickSnapshotId == null, + "Cannot set the current snapshot ID and cherry-pick snapshot changes"); + + if (setSnapshotId != null) { + long newSnapshotId = Long.parseLong(setSnapshotId); + table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); + } + + // if updating the table snapshot, perform that update first in case it fails + if (cherrypickSnapshotId != null) { + long newSnapshotId = Long.parseLong(cherrypickSnapshotId); + table.manageSnapshots().cherrypick(newSnapshotId).commit(); + } + } + + /** + * Applies a list of Flink table changes to an {@link UpdateSchema} operation. + * + * @param pendingUpdate an uncommitted UpdateSchema operation to configure + * @param schemaChanges a list of Flink table changes + */ + public static void applySchemaChanges( + UpdateSchema pendingUpdate, List schemaChanges) { + for (TableChange change : schemaChanges) { + if (change instanceof TableChange.AddColumn) { + TableChange.AddColumn addColumn = (TableChange.AddColumn) change; + Column flinkColumn = addColumn.getColumn(); + Preconditions.checkArgument( + FlinkCompatibilityUtil.isPhysicalColumn(flinkColumn), + "Unsupported table change: Adding computed column %s.", + flinkColumn.getName()); + Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); + if (flinkColumn.getDataType().getLogicalType().isNullable()) { + pendingUpdate.addColumn(flinkColumn.getName(), icebergType); + } else { + pendingUpdate.addRequiredColumn(flinkColumn.getName(), icebergType); + } + } else if (change instanceof TableChange.ModifyColumn) { + TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; + applyModifyColumn(pendingUpdate, modifyColumn); + } else if (change instanceof TableChange.DropColumn) { + TableChange.DropColumn dropColumn = (TableChange.DropColumn) change; + pendingUpdate.deleteColumn(dropColumn.getColumnName()); + } else if (change instanceof TableChange.AddWatermark) { + throw new UnsupportedOperationException("Unsupported table change: AddWatermark."); + } else if (change instanceof TableChange.ModifyWatermark) { + throw new UnsupportedOperationException("Unsupported table change: ModifyWatermark."); + } else if (change instanceof TableChange.DropWatermark) { + throw new UnsupportedOperationException("Unsupported table change: DropWatermark."); + } else if (change instanceof TableChange.AddUniqueConstraint) { + TableChange.AddUniqueConstraint addPk = (TableChange.AddUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, addPk.getConstraint()); + } else if (change instanceof TableChange.ModifyUniqueConstraint) { + TableChange.ModifyUniqueConstraint modifyPk = (TableChange.ModifyUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, modifyPk.getNewConstraint()); + } else if (change instanceof TableChange.DropConstraint) { + throw new UnsupportedOperationException("Unsupported table change: DropConstraint."); + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + } + + /** + * Applies a list of Flink table property changes to an {@link UpdateProperties} operation. + * + * @param pendingUpdate an uncommitted UpdateProperty operation to configure + * @param propertyChanges a list of Flink table changes + */ + public static void applyPropertyChanges( + UpdateProperties pendingUpdate, List propertyChanges) { + for (TableChange change : propertyChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption setOption = (TableChange.SetOption) change; + pendingUpdate.set(setOption.getKey(), setOption.getValue()); + } else if (change instanceof TableChange.ResetOption) { + TableChange.ResetOption resetOption = (TableChange.ResetOption) change; + pendingUpdate.remove(resetOption.getKey()); + } else { + throw new UnsupportedOperationException( + "The given table change is not a property change: " + change); + } + } + } + + private static void applyModifyColumn( + UpdateSchema pendingUpdate, TableChange.ModifyColumn modifyColumn) { + if (modifyColumn instanceof TableChange.ModifyColumnName) { + TableChange.ModifyColumnName modifyName = (TableChange.ModifyColumnName) modifyColumn; + pendingUpdate.renameColumn(modifyName.getOldColumnName(), modifyName.getNewColumnName()); + } else if (modifyColumn instanceof TableChange.ModifyColumnPosition) { + TableChange.ModifyColumnPosition modifyPosition = + (TableChange.ModifyColumnPosition) modifyColumn; + applyModifyColumnPosition(pendingUpdate, modifyPosition); + } else if (modifyColumn instanceof TableChange.ModifyPhysicalColumnType) { + TableChange.ModifyPhysicalColumnType modifyType = + (TableChange.ModifyPhysicalColumnType) modifyColumn; + Type type = FlinkSchemaUtil.convert(modifyType.getNewType().getLogicalType()); + String columnName = modifyType.getOldColumn().getName(); + pendingUpdate.updateColumn(columnName, type.asPrimitiveType()); + if (modifyType.getNewColumn().getDataType().getLogicalType().isNullable()) { + pendingUpdate.makeColumnOptional(columnName); + } else { + pendingUpdate.requireColumn(columnName); + } + } else if (modifyColumn instanceof TableChange.ModifyColumnComment) { + TableChange.ModifyColumnComment modifyComment = + (TableChange.ModifyColumnComment) modifyColumn; + pendingUpdate.updateColumnDoc( + modifyComment.getOldColumn().getName(), modifyComment.getNewComment()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column change: " + modifyColumn); + } + } + + private static void applyModifyColumnPosition( + UpdateSchema pendingUpdate, TableChange.ModifyColumnPosition modifyColumnPosition) { + TableChange.ColumnPosition newPosition = modifyColumnPosition.getNewPosition(); + if (newPosition instanceof TableChange.First) { + pendingUpdate.moveFirst(modifyColumnPosition.getOldColumn().getName()); + } else if (newPosition instanceof TableChange.After) { + TableChange.After after = (TableChange.After) newPosition; + pendingUpdate.moveAfter(modifyColumnPosition.getOldColumn().getName(), after.column()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column-position change: " + modifyColumnPosition); + } + } + + private static void applyUniqueConstraint( + UpdateSchema pendingUpdate, UniqueConstraint constraint) { + switch (constraint.getType()) { + case PRIMARY_KEY: + pendingUpdate.setIdentifierFields(constraint.getColumns()); + break; + case UNIQUE_KEY: + throw new UnsupportedOperationException( + "Unsupported table change: setting unique key constraints."); + default: + throw new UnsupportedOperationException( + "Cannot apply unknown unique constraint: " + constraint.getType().name()); + } + } +} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java similarity index 91% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java index 2c5c587f4ebf..f02af894e82b 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.catalog.Column; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; @@ -39,4 +40,8 @@ public static TypeInformation toTypeInfo(RowType rowType) { public static boolean isPhysicalColumn(TableColumn column) { return column.isPhysical(); } + + public static boolean isPhysicalColumn(Column column) { + return column.isPhysical(); + } } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java similarity index 100% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java diff --git a/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink/v1.15/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkCatalogTestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java index c3f8bf92e47a..ba08b76dd59d 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -85,6 +85,7 @@ public void testCreateCatalogCustomWithHiveCatalogTypeSet() { props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + Assertions.assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(IllegalArgumentException.class) @@ -96,6 +97,7 @@ public void testCreateCatalogCustomWithHiveCatalogTypeSet() { public void testLoadCatalogUnknown() { String catalogName = "unknownCatalog"; props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); + Assertions.assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(UnsupportedOperationException.class) diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java similarity index 53% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 472cf70b44ba..8f5ddde91851 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -28,6 +28,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.constraints.UniqueConstraint; @@ -105,9 +106,11 @@ public void testRenameTable() { new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); sql("ALTER TABLE tl RENAME TO tl2"); + Assertions.assertThatThrownBy(() -> getTableEnv().from("tl")) .isInstanceOf(ValidationException.class) .hasMessage("Table `tl` was not found."); + Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); Assert.assertEquals(tableSchema.asStruct(), actualSchema.asStruct()); } @@ -273,6 +276,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Excep Table table = table("tl"); TableOperations ops = ((BaseTable) table).operations(); Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) .rootCause() .isInstanceOf(IllegalArgumentException.class) @@ -294,7 +298,7 @@ public void testLoadTransformPartitionTable() throws TableNotExistException { } @Test - public void testAlterTable() throws TableNotExistException { + public void testAlterTableProperties() throws TableNotExistException { sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); Map properties = Maps.newHashMap(); properties.put("oldK", "oldV"); @@ -310,39 +314,297 @@ public void testAlterTable() throws TableNotExistException { assertThat(table("tl").properties()).containsAllEntriesOf(properties); // remove property - CatalogTable catalogTable = catalogTable("tl"); + sql("ALTER TABLE tl RESET('oldK')"); properties.remove("oldK"); - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .alterTable(new ObjectPath(DATABASE, "tl"), catalogTable.copy(properties), false); assertThat(table("tl").properties()).containsAllEntriesOf(properties); } @Test - public void testAlterTableWithPrimaryKey() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT, PRIMARY KEY(id) NOT ENFORCED) WITH ('oldK'='oldV')"); - Map properties = Maps.newHashMap(); - properties.put("oldK", "oldV"); + public void testAlterTableAddColumn() { + sql("CREATE TABLE tl(id BIGINT)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + schemaBefore.asStruct()); - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); + sql("ALTER TABLE tl ADD (dt STRING)"); + Schema schemaAfter1 = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfter1.asStruct()); - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); + // Add multiple columns + sql("ALTER TABLE tl ADD (col1 STRING, col2 BIGINT)"); + Schema schemaAfter2 = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct(), + schemaAfter2.asStruct()); - // remove property - CatalogTable catalogTable = catalogTable("tl"); - properties.remove("oldK"); - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .alterTable(new ObjectPath(DATABASE, "tl"), catalogTable.copy(properties), false); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); + // Adding a required field should fail because Iceberg's SchemaUpdate does not allow + // incompatible changes. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); + + // Adding an existing field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Try to add a column `id` which already exists in the table."); + } + + @Test + public void testAlterTableDropColumn() { + sql("CREATE TABLE tl(id BIGINT, dt STRING, col1 STRING, col2 BIGINT)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl DROP (dt)"); + Schema schemaAfter1 = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct(), + schemaAfter1.asStruct()); + + // Drop multiple columns + sql("ALTER TABLE tl DROP (col1, col2)"); + Schema schemaAfter2 = table("tl").schema(); + Assert.assertEquals( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct(), + schemaAfter2.asStruct()); + + // Dropping an non-existing field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `foo` does not exist in the base table."); + + // Dropping an already-deleted field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `dt` does not exist in the base table."); + } + + @Test + public void testAlterTableModifyColumnName() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl RENAME dt TO data"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())) + .asStruct(), + schemaAfter.asStruct()); + } + + @Test + public void testAlterTableModifyColumnType() { + sql("CREATE TABLE tl(id INTEGER, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + // Promote type from Integer to Long + sql("ALTER TABLE tl MODIFY (id BIGINT)"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfter.asStruct()); + + // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's + // validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column type: dt: string -> int"); + } + + @Test + public void testAlterTableModifyColumnNullability() { + sql("CREATE TABLE tl(id INTEGER NOT NULL, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + // Changing nullability from optional to required should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); + + // Set nullability from required to optional + sql("ALTER TABLE tl MODIFY (id INTEGER)"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfter.asStruct()); + } + + @Test + public void testAlterTableModifyColumnPosition() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING FIRST)"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(1, "id", Types.LongType.get())) + .asStruct(), + schemaAfter.asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING AFTER id)"); + Schema schemaAfterAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaAfterAfter.asStruct()); + + // Modifying the position of a non-existing column should fail due to Flink's internal + // validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Try to modify a column `non_existing` which does not exist in the table."); + + // Moving a column after a non-existing column should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Referenced column `non_existing` by 'AFTER' does not exist in the table."); + } + + @Test + public void testAlterTableModifyColumnComment() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING COMMENT 'comment for dt field')"); + Schema schemaAfter = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get(), "comment for dt field")) + .asStruct(), + schemaAfter.asStruct()); + } + + @Test + public void testAlterTableConstraint() { + sql("CREATE TABLE tl(id BIGINT NOT NULL, dt STRING NOT NULL, col1 STRING)"); + Schema schemaBefore = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct(), + schemaBefore.asStruct()); + Assert.assertEquals(ImmutableSet.of(), schemaBefore.identifierFieldNames()); + + sql("ALTER TABLE tl ADD (PRIMARY KEY (id) NOT ENFORCED)"); + Schema schemaAfterAdd = table("tl").schema(); + Assert.assertEquals(ImmutableSet.of("id"), schemaAfterAdd.identifierFieldNames()); + + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (dt) NOT ENFORCED)"); + Schema schemaAfterModify = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct(), + schemaAfterModify.asStruct()); + Assert.assertEquals(ImmutableSet.of("dt"), schemaAfterModify.identifierFieldNames()); + + // Composite primary key + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, dt) NOT ENFORCED)"); + Schema schemaAfterComposite = table("tl").schema(); + Assert.assertEquals( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct(), + schemaAfterComposite.asStruct()); + Assert.assertEquals(ImmutableSet.of("id", "dt"), schemaAfterComposite.identifierFieldNames()); + + // Setting an optional field as primary key should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Setting a composite key containing an optional field should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Dropping constraints is not supported yet + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(UnsupportedOperationException.class) + .hasRootCauseMessage("Unsupported table change: DropConstraint."); } @Test diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java similarity index 97% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index fad65f4c63c8..0008e4320c8a 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -92,7 +92,8 @@ public void testListPartitionsWithUnpartitionedTable() { FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) .isInstanceOf(TableNotPartitionedException.class) - .hasMessage("Table " + objectPath + " in catalog " + catalogName + " is not partitioned."); + .hasMessageStartingWith("Table db.test_table in catalog") + .hasMessageEndingWith("is not partitioned."); } @Test diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java index 16b220ba6715..4ac32c08ebef 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -407,9 +407,10 @@ public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { Types.StructType.of( Types.NestedField.required(2, "inner", Types.IntegerType.get())))), Sets.newHashSet(2)); + Assertions.assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Could not create a PRIMARY KEY") - .hasMessageContaining("Column 'struct.inner' does not exist"); + .hasMessageContaining("Column 'struct.inner' does not exist."); } } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java similarity index 96% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 3e574b841c8d..7d3777935ca8 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -416,7 +416,7 @@ private static void assertAvroEquals( Collection expectedArrayData = (Collection) expected; ArrayData actualArrayData; try { - actualArrayData = (ArrayData) actual; + actualArrayData = convertToArray(actual); } catch (ClassCastException e) { actualArrayData = new GenericArrayData((Object[]) actual); } @@ -462,6 +462,29 @@ private static void assertAvroEquals( } } + private static GenericArrayData convertToArray(Object actual) { + Class arrayClass = actual.getClass(); + if (Object[].class.equals(arrayClass)) { + return new GenericArrayData((Object[]) actual); + } else if (int[].class.equals(arrayClass)) { + return new GenericArrayData((int[]) actual); + } else if (long[].class.equals(arrayClass)) { + return new GenericArrayData((long[]) actual); + } else if (float[].class.equals(arrayClass)) { + return new GenericArrayData((float[]) actual); + } else if (double[].class.equals(arrayClass)) { + return new GenericArrayData((double[]) actual); + } else if (short[].class.equals(arrayClass)) { + return new GenericArrayData((short[]) actual); + } else if (byte[].class.equals(arrayClass)) { + return new GenericArrayData((byte[]) actual); + } else if (boolean[].class.equals(arrayClass)) { + return new GenericArrayData((boolean[]) actual); + } else { + throw new IllegalArgumentException("Unsupported type " + arrayClass); + } + } + private static void assertArrayValues( Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { List expectedElements = Lists.newArrayList(expectedArray); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java diff --git a/flink/v1.15/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 similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index d9d57fb7107e..14f12422da96 100644 --- a/flink/v1.15/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 @@ -138,7 +138,7 @@ public void testCompressionParquet() throws Exception { resultProperties.get(TableProperties.PARQUET_COMPRESSION)); Assert.assertEquals( TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT)); + resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); } else { Assert.assertEquals( initProperties.get(TableProperties.PARQUET_COMPRESSION), diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java diff --git a/flink/v1.15/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 similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 4bef4d4244e5..11a73d2cc144 100644 --- a/flink/v1.15/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 @@ -197,6 +197,7 @@ public void testJobHashDistributionMode() { .updateProperties() .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) .commit(); + Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Flink does not support 'range' write distribution mode now."); @@ -344,6 +345,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .tableLoader(tableLoader) .writeParallelism(parallelism) .setAll(newProps); + Assertions.assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); @@ -363,6 +365,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .tableLoader(tableLoader) .writeParallelism(parallelism) .setAll(newProps); + Assertions.assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file format: UNRECOGNIZED"); diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 507df9e35215..0b403756cefe 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -244,6 +244,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { .hasMessageStartingWith( "In 'hash' distribution mode with equality fields set, partition field") .hasMessageContaining("should be included in equality fields:"); + } else { testChangeLogs( ImmutableList.of("id"), @@ -287,7 +288,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { expectedRecords, branch)) .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("should be included in equality fields"); + .hasMessageContaining("should be included in equality fields:"); } } diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java diff --git a/flink/v1.15/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 similarity index 56% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index dd7fcafe5336..890cc361b246 100644 --- a/flink/v1.15/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 @@ -21,41 +21,43 @@ 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.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.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() { - GenericRowData rowDataA = GenericRowData.of(StringData.fromString("a")); - GenericRowData rowDataB = GenericRowData.of(StringData.fromString("b")); + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>( - 1, - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType())))); + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataA); - mapDataStatistics1.add(rowDataB); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(rowDataA); + mapDataStatistics2.add(keyA); aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataA)) + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataA) - + mapDataStatistics2.statistics().get(rowDataA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(rowDataB)) + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) .isEqualTo( - mapDataStatistics1.statistics().get(rowDataB) - + mapDataStatistics2.statistics().getOrDefault(rowDataB, 0L)); + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); } } diff --git a/flink/v1.15/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 similarity index 67% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 48e4e4d8f966..4c64ce522201 100644 --- a/flink/v1.15/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 @@ -21,32 +21,33 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -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.types.Types; import org.junit.Before; import org.junit.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; - private final RowType rowType = RowType.of(new VarCharType()); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private AggregatedStatisticsTracker> + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> aggregatedStatisticsTracker; + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + @Before public void before() throws Exception { aggregatedStatisticsTracker = @@ -56,8 +57,8 @@ public void before() throws Exception { @Test public void receiveNewerDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -67,8 +68,8 @@ public void receiveNewerDataStatisticEvent() { assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -82,10 +83,10 @@ public void receiveNewerDataStatisticEvent() { @Test public void receiveOlderDataStatisticEventTest() { MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - checkpoint2Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint3Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -94,8 +95,8 @@ public void receiveOlderDataStatisticEventTest() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return @@ -110,10 +111,10 @@ public void receiveOlderDataStatisticEventTest() { @Test public void receiveCompletedDataStatisticEvent() { MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -122,14 +123,14 @@ public void receiveCompletedDataStatisticEvent() { .isNull(); MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = + AggregatedStatistics> completedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion( 1, checkpoint1Subtask1DataStatisticEvent); @@ -137,20 +138,20 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics globalDataStatistics = (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataA)) + assertThat((long) globalDataStatistics.statistics().get(keyA)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA)); - assertThat((long) globalDataStatistics.statistics().get(binaryRowDataB)) + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB)); + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) .isEqualTo(completedStatistics.checkpointId() + 1); MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(binaryRowDataA); - DataStatisticsEvent> + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> checkpoint2Subtask0DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); assertThat( @@ -160,8 +161,8 @@ public void receiveCompletedDataStatisticEvent() { assertThat(completedStatistics.checkpointId()).isEqualTo(1); MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(binaryRowDataB); - DataStatisticsEvent> + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> checkpoint2Subtask1DataStatisticEvent = DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); // Receive data statistics from all subtasks at checkpoint 2 diff --git a/flink/v1.15/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 similarity index 60% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 3be12ea1965b..3df714059c37 100644 --- a/flink/v1.15/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 @@ -24,19 +24,15 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import org.apache.flink.api.common.typeutils.TypeSerializer; 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.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.binary.BinaryRowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; 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; @@ -44,20 +40,21 @@ 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 TypeSerializer>> - statisticsSerializer; + + 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> + private DataStatisticsCoordinator> dataStatisticsCoordinator; @Before public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - statisticsSerializer = - MapDataStatisticsSerializer.fromKeySerializer( - new RowDataSerializer(RowType.of(new VarCharType()))); - dataStatisticsCoordinator = new DataStatisticsCoordinator<>( OPERATOR_NAME, @@ -77,11 +74,12 @@ public void testThrowExceptionWhenNotStarted() { assertThatThrownBy( () -> dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) .isInstanceOf(IllegalStateException.class) .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.subtaskFailed(0, null)) + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) .isInstanceOf(IllegalStateException.class) .hasMessage(failureMessage); assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) @@ -92,67 +90,75 @@ public void testThrowExceptionWhenNotStarted() { @Test public void testDataStatisticsEventHandling() throws Exception { tasksReady(); - // When coordinator handles events from operator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - RowType rowType = RowType.of(new VarCharType()); - BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); + SortKey key = sortKey.copy(); MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(binaryRowDataA); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataB); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - checkpoint1Subtask0DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask0DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(binaryRowDataA); - checkpoint1Subtask1DataStatistic.add(binaryRowDataB); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - checkpoint1Subtask1DataStatistic.add(binaryRowDataC); - DataStatisticsEvent> + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> checkpoint1Subtask1DataStatisticEvent = DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, checkpoint1Subtask1DataStatisticEvent); + 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( - binaryRowDataA, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataA) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataA), - binaryRowDataB, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataB) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataB), - binaryRowDataC, - checkpoint1Subtask0DataStatistic.statistics().get(binaryRowDataC) - + (long) checkpoint1Subtask1DataStatistic.statistics().get(binaryRowDataC))); + keyA, 2L, + keyB, 3L, + keyC, 5L)); } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator> dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { - dataStatisticsCoordinator.subtaskReady(i, receivingTasks.createGatewayForSubtask(i)); + dataStatisticsCoordinator.executionAttemptReady( + i, 0, receivingTasks.createGatewayForSubtask(i, 0)); } } static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + DataStatisticsCoordinator> coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..5e0a752be506 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsCoordinatorProvider { + private static final OperatorID OPERATOR_ID = new OperatorID(); + private static final int NUM_SUBTASKS = 1; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; + private EventReceivingTasks receivingTasks; + + @Before + public void before() { + provider = + new DataStatisticsCoordinatorProvider<>( + "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + @Test + @SuppressWarnings("unchecked") + public void testCheckpointAndReset() throws Exception { + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + SortKey keyD = sortKey.copy(); + keyD.set(0, "c"); + SortKey keyE = sortKey.copy(); + keyE.set(0, "c"); + + try (RecreateOnResetOperatorCoordinator coordinator = + (RecreateOnResetOperatorCoordinator) + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + + // Handle events from operators for checkpoint 1 + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics + MapDataStatistics checkpoint1GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint1GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); + + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } + } + + private byte[] waitForCheckpoint( + long checkpointId, + DataStatisticsCoordinator> coordinator) + throws InterruptedException, ExecutionException { + CompletableFuture future = new CompletableFuture<>(); + coordinator.checkpointCoordinator(checkpointId, future); + return future.get(); + } +} diff --git a/flink/v1.15/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 similarity index 64% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 880cb3d5516f..0e99a2d74ccb 100644 --- a/flink/v1.15/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 @@ -27,7 +27,6 @@ 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.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -50,33 +49,37 @@ 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.binary.BinaryRowData; 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 RowType rowType = RowType.of(new VarCharType()); + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final GenericRowData genericRowDataA = GenericRowData.of(StringData.fromString("a")); - private final GenericRowData genericRowDataB = GenericRowData.of(StringData.fromString("b")); - // When operator hands events from coordinator, DataStatisticsUtil#deserializeDataStatistics - // deserializes bytes into BinaryRowData - private final BinaryRowData binaryRowDataA = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("a"))); - private final BinaryRowData binaryRowDataB = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("b"))); - private final BinaryRowData binaryRowDataC = - new RowDataSerializer(rowType).toBinaryRow(GenericRowData.of(StringData.fromString("c"))); - private final TypeSerializer>> - statisticsSerializer = MapDataStatisticsSerializer.fromKeySerializer(rowSerializer); - private DataStatisticsOperator> operator; + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; private Environment getTestingEnvironment() { return new StreamMockEnvironment( @@ -99,20 +102,10 @@ public void before() throws Exception { new MockOutput<>(Lists.newArrayList())); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator> createOperator() { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - KeySelector keySelector = - new KeySelector() { - private static final long serialVersionUID = 7662520075515707428L; - - @Override - public RowData getKey(RowData value) { - return value; - } - }; - return new DataStatisticsOperator<>( - "testOperator", keySelector, mockGateway, statisticsSerializer); + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } @After @@ -123,20 +116,26 @@ public void clean() throws Exception { @Test public void testProcessElement() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataA)); - operator.processElement(new StreamRecord<>(genericRowDataB)); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); + Map statsMap = mapDataStatistics.statistics(); assertThat(statsMap).hasSize(2); - assertThat(statsMap) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(genericRowDataA, 2L, genericRowDataB, 1L)); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + testHarness.endInput(); } } @@ -144,11 +143,14 @@ public void testProcessElement() throws Exception { @Test public void testOperatorOutput() throws Exception { try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness = createHarness(this.operator)) { - testHarness.processElement(new StreamRecord<>(genericRowDataA)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); - testHarness.processElement(new StreamRecord<>(genericRowDataB)); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); List recordsOutput = testHarness.extractOutputValues().stream() @@ -157,7 +159,10 @@ public void testOperatorOutput() throws Exception { .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( - ImmutableList.of(genericRowDataA, genericRowDataB, genericRowDataB)); + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); } } @@ -165,36 +170,61 @@ public void testOperatorOutput() throws Exception { public void testRestoreState() throws Exception { OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness1 = createHarness(this.operator)) { - DataStatistics> mapDataStatistics = - new MapDataStatistics(); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataA); - mapDataStatistics.add(binaryRowDataB); - mapDataStatistics.add(binaryRowDataC); - operator.handleOperatorEvent( - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer)); + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + .containsExactlyInAnyOrderEntriesOf(expectedMap); snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = + DataStatisticsOperator> restoredOperator = createOperator(); try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(restoredOperator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of(binaryRowDataA, 2L, binaryRowDataB, 1L, binaryRowDataC, 1L)); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); } } @@ -209,18 +239,16 @@ private StateInitializationContext getStateContext() throws Exception { } private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> createHarness( - final DataStatisticsOperator> + final DataStatisticsOperator> dataStatisticsOperator) throws Exception { OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> + RowData, DataStatisticsOrRecord>> harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup( - new DataStatisticsOrRecordSerializer<>( - MapDataStatisticsSerializer.fromKeySerializer(rowSerializer), rowSerializer)); + harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); harness.open(); return harness; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java diff --git a/flink/v1.15/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 similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index b4301b2fc4d7..b537efa7272b 100644 --- a/flink/v1.15/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 @@ -354,6 +354,7 @@ public void testIncrementalReadViaTag() throws Exception { .buildOrThrow()), expected, TestFixtures.SCHEMA); + Assertions.assertThatThrownBy( () -> runWithOptions( @@ -364,6 +365,7 @@ public void testIncrementalReadViaTag() throws Exception { .buildOrThrow())) .isInstanceOf(Exception.class) .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + Assertions.assertThatThrownBy( () -> runWithOptions( diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java diff --git a/flink/v1.15/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 similarity index 90% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index c3e23d34faf5..ff14bc406276 100644 --- a/flink/v1.15/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 @@ -96,15 +96,15 @@ public void before() { public void clean() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - sql("DROP CATALOG IF EXISTS %s", CATALOG_NAME); + dropCatalog(CATALOG_NAME, true); } @Test public void testLimitPushDown() { + Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) - .as("Invalid limit number: -1 ") .isInstanceOf(SqlParserException.class) - .hasMessageContaining("SQL parse failed. Encountered \"-\""); + .hasMessageStartingWith("SQL parse failed."); Assert.assertEquals( "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); @@ -420,8 +420,12 @@ public void testFilterPushDownInNull() { Assert.assertEquals("Should have 1 record", 1, result.size()); Assert.assertEquals( "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + + // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it + // and push the rest down. + String expectedScan = "ref(name=\"data\") == \"iceberg\""; Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); } @Test @@ -443,8 +447,9 @@ public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + Assert.assertNull( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.", + lastScanEvent); } @Test @@ -540,6 +545,17 @@ public void testFilterPushDownLike() { Assert.assertEquals("Should create only one scan", 1, scanEventCount); Assert.assertEquals( "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + + // %% won't match the row with null value + sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; + resultLike = sql(sqlLike); + Assert.assertEquals("Should have 2 records", 2, resultLike.size()); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedRecords, resultLike); + String expectedScan = "not_null(ref(name=\"data\"))"; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); } @Test @@ -547,7 +563,7 @@ public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; List resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 0, resultLike.size()); + Assert.assertEquals("Should have 0 record", 0, resultLike.size()); Assert.assertEquals( "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); @@ -565,15 +581,6 @@ public void testFilterNotPushDownLike() { Assert.assertEquals( "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 3 records", 3, resultLike.size()); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, resultLike); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; resultLike = sql(sqlNoPushDown); Assert.assertEquals("Should have 1 record", 1, resultLike.size()); @@ -600,47 +607,8 @@ public void testFilterPushDown2Literal() { "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); } - /** - * NaN is not supported by flink now, so we add the test case to assert the parse error, when we - * upgrade the flink that supports NaN, we will delele the method, and add some test case to test - * NaN. - */ - @Test - public void testSqlParseError() { - String sqlParseErrorEqual = - String.format("SELECT * FROM %s WHERE d = CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorEqual)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorNotEqual = - String.format("SELECT * FROM %s WHERE d <> CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorNotEqual)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorGT = - String.format("SELECT * FROM %s WHERE d > CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorGT)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorLT = - String.format("SELECT * FROM %s WHERE d < CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorLT)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorGTE = - String.format("SELECT * FROM %s WHERE d >= CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorGTE)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); - - String sqlParseErrorLTE = - String.format("SELECT * FROM %s WHERE d <= CAST('NaN' AS DOUBLE) ", TABLE_NAME); - Assertions.assertThatThrownBy(() -> sql(sqlParseErrorLTE)) - .isInstanceOf(NumberFormatException.class) - .hasMessageContaining("Infinite or NaN"); + @Test + public void testSqlParseNaN() { + // todo add some test case to test NaN } } diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java diff --git a/flink/v1.15/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 similarity index 91% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 7547323871c1..aa4b9cd79e55 100644 --- a/flink/v1.15/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 @@ -30,7 +30,6 @@ import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; @@ -40,8 +39,8 @@ import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -51,10 +50,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; -import org.apache.flink.streaming.api.operators.collect.CollectResultIterator; -import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator; -import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory; -import org.apache.flink.streaming.api.operators.collect.CollectStreamSink; import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; @@ -98,7 +93,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); @@ -304,14 +302,8 @@ public void testThrottling() throws Exception { SOURCE_NAME, TypeInformation.of(RowData.class)); - // Flink 1.15 only change - start - CollectResultIterator resultStream = addCollectSink(stream); - - // Start the job - JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - resultStream.setJobClient(jobClient); - try (CollectResultIterator resultIterator = resultStream) { - // Flink 1.15 only change - end + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); // Check that the read the non-blocked data // The first RECORD_NUM_FOR_2_SPLITS should be read @@ -460,22 +452,4 @@ public long extractTimestamp(RowData element, long recordTimestamp) { return element.getTimestamp(0, 0).getMillisecond(); } } - - // Flink 1.15 only method - private CollectResultIterator addCollectSink(DataStream stream) { - TypeSerializer serializer = - stream.getType().createSerializer(stream.getExecutionConfig()); - String accumulatorName = "dataStreamCollect_" + UUID.randomUUID(); - CollectSinkOperatorFactory factory = - new CollectSinkOperatorFactory<>(serializer, accumulatorName); - CollectSinkOperator operator = (CollectSinkOperator) factory.getOperator(); - CollectStreamSink sink = new CollectStreamSink<>(stream, factory); - sink.name("Data stream collect sink"); - stream.getExecutionEnvironment().addOperator(sink.getTransformation()); - return new CollectResultIterator<>( - operator.getOperatorIdFuture(), - serializer, - accumulatorName, - stream.getExecutionEnvironment().getCheckpointConfig()); - } } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java diff --git a/flink/v1.15/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 similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java diff --git a/flink/v1.15/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 similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index eaba615ce910..633e11718b9b 100644 --- a/flink/v1.15/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 @@ -219,6 +219,7 @@ public void testConsumeFilesWithBranch() throws Exception { Row row1 = Row.of(1, "aaa", "2021-01-01"); Row row2 = Row.of(2, "bbb", "2021-01-01"); insertRows(table, row1, row2); + Assertions.assertThatThrownBy( () -> exec( @@ -304,6 +305,7 @@ public void testConsumeFromStartTag() throws Exception { assertRows(ImmutableList.of(row7), iterator); } result.getJobClient().ifPresent(JobClient::cancel); + Assertions.assertThatThrownBy( () -> exec( diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 8af1dd883f4c..494c633088d9 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -250,6 +250,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .monitorInterval(Duration.ofMillis(100)) .maxPlanningSnapshotCount(0) .build(); + Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -259,6 +260,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .monitorInterval(Duration.ofMillis(100)) .maxPlanningSnapshotCount(-10) .build(); + Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index bb747caae589..1bb2b1c6bf85 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -337,6 +337,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); @@ -362,6 +363,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); @@ -423,6 +425,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find a snapshot after: 1"); @@ -443,9 +446,10 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); + .hasMessageStartingWith("Cannot find a snapshot after:"); } @Test diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java similarity index 99% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java similarity index 94% rename from flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java rename to flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index caacbd4b5a30..a805d160d809 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -26,6 +26,6 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - Assert.assertEquals("1.15.0", FlinkPackage.version()); + Assert.assertEquals("1.18.0", FlinkPackage.version()); } } diff --git a/flink/v1.15/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.15/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/format/spec.md b/format/spec.md index 5d6dded5ee76..80cdd6d2987f 100644 --- a/format/spec.md +++ b/format/spec.md @@ -963,7 +963,7 @@ Note that the string map case is for maps where the key type is a string. Using **Data Type Mappings** -Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required. +Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required to be stored as [field IDs](http://github.com/apache/parquet-format/blob/40699d05bd24181de6b1457babbee2c16dce3803/src/main/thrift/parquet.thrift#L459) on the parquet schema. Lists must use the [3-level representation](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists). diff --git a/gradle.properties b/gradle.properties index 1bce26270354..84644da7c4ba 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.17 -systemProp.knownFlinkVersions=1.15,1.16,1.17 +systemProp.defaultFlinkVersions=1.18 +systemProp.knownFlinkVersions=1.16,1.17,1.18 systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d9ca188e9384..1ddaccb7a174 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,19 +27,19 @@ arrow = "14.0.1" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" -awssdk-bom = "2.21.29" +awssdk-bom = "2.21.42" azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" delta-standalone = "0.6.0" -delta-core = "2.2.0" +delta-spark = "3.0.0" esotericsoftware-kryo = "4.0.2" errorprone-annotations = "2.23.0" findbugs-jsr305 = "3.0.2" -flink115 = { strictly = "[1.15, 1.16[", prefer = "1.15.0"} # see rich version usage explanation above flink116 = { strictly = "[1.16, 1.17[", prefer = "1.16.2"} flink117 = { strictly = "[1.17, 1.18[", prefer = "1.17.1"} -google-libraries-bom = "26.27.0" +flink118 = { strictly = "[1.18, 1.19[", prefer = "1.18.0"} +google-libraries-bom = "26.28.0" guava = "32.1.3-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" @@ -75,10 +75,10 @@ roaringbitmap = "1.0.0" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.14.3" +snowflake-jdbc = "3.14.4" spark-hive32 = "3.2.2" -spark-hive33 = "3.3.2" -spark-hive34 = "3.4.1" +spark-hive33 = "3.3.3" +spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" @@ -104,12 +104,6 @@ calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "ca delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } -flink115-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink115" } -flink115-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink115" } -flink115-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink115" } -flink115-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink115" } -flink115-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink115" } -flink115-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink115" } flink116-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink116" } flink116-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink116" } flink116-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink116" } @@ -122,6 +116,12 @@ flink117-connector-files = { module = "org.apache.flink:flink-connector-files", 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" } +flink118-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink118" } +flink118-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink118" } +flink118-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink118" } 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" } @@ -169,13 +169,8 @@ snowflake-jdbc = { module = "net.snowflake:snowflake-jdbc", version.ref = "snowf # test libraries assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-core" } awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } -delta-core = { module = "io.delta:delta-core_2.12", version.ref = "delta-core" } +delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } -flink115-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink115" } -flink115-core = { module = "org.apache.flink:flink-core", version.ref = "flink115" } -flink115-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink115" } -flink115-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink115" } -flink115-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink115" } flink116-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink116" } flink116-core = { module = "org.apache.flink:flink-core", version.ref = "flink116" } flink116-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink116" } @@ -186,6 +181,11 @@ flink117-core = { module = "org.apache.flink:flink-core", version.ref = "flink11 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" } +flink118-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink118" } +flink118-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink118" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jackson-annotations = { module = "com.fasterxml.jackson.core:jackson-annotations", version.ref = "jackson-annotations" } jackson-dataformat-xml = { module = "com.fasterxml.jackson.dataformat:jackson-dataformat-xml", version.ref = "jackson-dataformat-xml" } diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java index 84d11d03a741..1592a3461b40 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -147,7 +148,9 @@ public void testSchemaConvertToIcebergSchemaForEveryPrimitiveType() { public void testNotSupportedTypes() { for (FieldSchema notSupportedField : getNotSupportedFieldSchemas()) { assertThatThrownBy( - () -> HiveSchemaUtil.convert(Lists.newArrayList(Arrays.asList(notSupportedField)))) + () -> + HiveSchemaUtil.convert( + Lists.newArrayList(Collections.singletonList(notSupportedField)))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Unsupported Hive type"); } @@ -197,7 +200,7 @@ public void testConversionWithoutLastComment() { Arrays.asList( TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.BIGINT_TYPE_NAME), TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.STRING_TYPE_NAME)), - Arrays.asList("customer comment")); + Collections.singletonList("customer comment")); assertThat(schema.asStruct()).isEqualTo(expected.asStruct()); } diff --git a/mr/build.gradle b/mr/build.gradle index 848f6dee3c43..37e665d6d2e6 100644 --- a/mr/build.gradle +++ b/mr/build.gradle @@ -23,6 +23,9 @@ project(':iceberg-mr') { exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' } } + test { + useJUnitPlatform() + } dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java b/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java index f849163acc6a..012ad2350224 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestCatalogs.java @@ -20,8 +20,10 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.Optional; import java.util.Properties; import org.apache.hadoop.conf.Configuration; @@ -39,12 +41,9 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestCatalogs { @@ -54,9 +53,9 @@ public class TestCatalogs { private Configuration conf; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Before + @BeforeEach public void before() { conf = new Configuration(); } @@ -65,25 +64,25 @@ public void before() { public void testLoadTableFromLocation() throws IOException { conf.set(CatalogUtil.ICEBERG_CATALOG_TYPE, Catalogs.LOCATION); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf)) + assertThatThrownBy(() -> Catalogs.loadTable(conf)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Table location not set"); HadoopTables tables = new HadoopTables(); - Table hadoopTable = tables.create(SCHEMA, temp.newFolder("hadoop_tables").toString()); + Table hadoopTable = tables.create(SCHEMA, temp.resolve("hadoop_tables").toString()); conf.set(InputFormatConfig.TABLE_LOCATION, hadoopTable.location()); - Assert.assertEquals(hadoopTable.location(), Catalogs.loadTable(conf).location()); + assertThat(Catalogs.loadTable(conf).location()).isEqualTo(hadoopTable.location()); } @Test public void testLoadTableFromCatalog() throws IOException { String defaultCatalogName = "default"; - String warehouseLocation = temp.newFolder("hadoop", "warehouse").toString(); + String warehouseLocation = temp.resolve("hadoop").resolve("warehouse").toString(); setCustomCatalogProperties(defaultCatalogName, warehouseLocation); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf)) + assertThatThrownBy(() -> Catalogs.loadTable(conf)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Table identifier not set"); @@ -92,15 +91,15 @@ public void testLoadTableFromCatalog() throws IOException { conf.set(InputFormatConfig.TABLE_IDENTIFIER, "table"); - Assert.assertEquals(hadoopCatalogTable.location(), Catalogs.loadTable(conf).location()); + assertThat(Catalogs.loadTable(conf).location()).isEqualTo(hadoopCatalogTable.location()); } @Test public void testCreateDropTableToLocation() throws IOException { Properties missingSchema = new Properties(); - missingSchema.put("location", temp.newFolder("hadoop_tables").toString()); + missingSchema.put("location", temp.resolve("hadoop_tables").toString()); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) .isInstanceOf(NullPointerException.class) .hasMessage("Table schema not set"); @@ -108,12 +107,12 @@ public void testCreateDropTableToLocation() throws IOException { Properties missingLocation = new Properties(); missingLocation.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(SCHEMA)); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingLocation)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingLocation)) .isInstanceOf(NullPointerException.class) .hasMessage("Table location not set"); Properties properties = new Properties(); - properties.put("location", temp.getRoot() + "/hadoop_tables"); + properties.put("location", temp.toFile() + "/hadoop_tables"); properties.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(SCHEMA)); properties.put(InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(SPEC)); properties.put("dummy", "test"); @@ -123,20 +122,21 @@ public void testCreateDropTableToLocation() throws IOException { HadoopTables tables = new HadoopTables(); Table table = tables.load(properties.getProperty("location")); - Assert.assertEquals(properties.getProperty("location"), table.location()); - Assert.assertEquals(SchemaParser.toJson(SCHEMA), SchemaParser.toJson(table.schema())); - Assert.assertEquals(PartitionSpecParser.toJson(SPEC), PartitionSpecParser.toJson(table.spec())); + assertThat(table.location()).isEqualTo(properties.getProperty("location")); + assertThat(SchemaParser.toJson(table.schema())).isEqualTo(SchemaParser.toJson(SCHEMA)); + assertThat(PartitionSpecParser.toJson(table.spec())) + .isEqualTo(PartitionSpecParser.toJson(SPEC)); assertThat(table.properties()).containsEntry("dummy", "test"); - Assertions.assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) + assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) .isInstanceOf(NullPointerException.class) .hasMessage("Table location not set"); Properties dropProperties = new Properties(); - dropProperties.put("location", temp.getRoot() + "/hadoop_tables"); + dropProperties.put("location", temp.toFile() + "/hadoop_tables"); Catalogs.dropTable(conf, dropProperties); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) + assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist at location: " + properties.getProperty("location")); } @@ -145,7 +145,7 @@ public void testCreateDropTableToLocation() throws IOException { public void testCreateDropTableToCatalog() throws IOException { TableIdentifier identifier = TableIdentifier.of("test", "table"); String defaultCatalogName = "default"; - String warehouseLocation = temp.newFolder("hadoop", "warehouse").toString(); + String warehouseLocation = temp.resolve("hadoop").resolve("warehouse").toString(); setCustomCatalogProperties(defaultCatalogName, warehouseLocation); @@ -153,14 +153,14 @@ public void testCreateDropTableToCatalog() throws IOException { missingSchema.put("name", identifier.toString()); missingSchema.put(InputFormatConfig.CATALOG_NAME, defaultCatalogName); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingSchema)) .isInstanceOf(NullPointerException.class) .hasMessage("Table schema not set"); Properties missingIdentifier = new Properties(); missingIdentifier.put(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(SCHEMA)); missingIdentifier.put(InputFormatConfig.CATALOG_NAME, defaultCatalogName); - Assertions.assertThatThrownBy(() -> Catalogs.createTable(conf, missingIdentifier)) + assertThatThrownBy(() -> Catalogs.createTable(conf, missingIdentifier)) .isInstanceOf(NullPointerException.class) .hasMessage("Table identifier not set"); @@ -176,11 +176,12 @@ public void testCreateDropTableToCatalog() throws IOException { HadoopCatalog catalog = new CustomHadoopCatalog(conf, warehouseLocation); Table table = catalog.loadTable(identifier); - Assert.assertEquals(SchemaParser.toJson(SCHEMA), SchemaParser.toJson(table.schema())); - Assert.assertEquals(PartitionSpecParser.toJson(SPEC), PartitionSpecParser.toJson(table.spec())); + assertThat(SchemaParser.toJson(table.schema())).isEqualTo(SchemaParser.toJson(SCHEMA)); + assertThat(PartitionSpecParser.toJson(table.spec())) + .isEqualTo(PartitionSpecParser.toJson(SPEC)); assertThat(table.properties()).containsEntry("dummy", "test"); - Assertions.assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) + assertThatThrownBy(() -> Catalogs.dropTable(conf, new Properties())) .isInstanceOf(NullPointerException.class) .hasMessage("Table identifier not set"); @@ -189,7 +190,7 @@ public void testCreateDropTableToCatalog() throws IOException { dropProperties.put(InputFormatConfig.CATALOG_NAME, defaultCatalogName); Catalogs.dropTable(conf, dropProperties); - Assertions.assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) + assertThatThrownBy(() -> Catalogs.loadTable(conf, dropProperties)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: test.table"); } @@ -198,11 +199,11 @@ public void testCreateDropTableToCatalog() throws IOException { public void testLoadCatalogDefault() { String catalogName = "barCatalog"; Optional defaultCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(defaultCatalog.isPresent()); - Assertions.assertThat(defaultCatalog.get()).isInstanceOf(HiveCatalog.class); + assertThat(defaultCatalog).isPresent(); + assertThat(defaultCatalog.get()).isInstanceOf(HiveCatalog.class); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertTrue(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isTrue(); } @Test @@ -212,11 +213,11 @@ public void testLoadCatalogHive() { InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE); Optional hiveCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(hiveCatalog.isPresent()); - Assertions.assertThat(hiveCatalog.get()).isInstanceOf(HiveCatalog.class); + assertThat(hiveCatalog).isPresent(); + assertThat(hiveCatalog.get()).isInstanceOf(HiveCatalog.class); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertTrue(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isTrue(); } @Test @@ -230,13 +231,13 @@ public void testLoadCatalogHadoop() { catalogName, CatalogProperties.WAREHOUSE_LOCATION), "/tmp/mylocation"); Optional hadoopCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(hadoopCatalog.isPresent()); - Assertions.assertThat(hadoopCatalog.get()).isInstanceOf(HadoopCatalog.class); - Assert.assertEquals( - "HadoopCatalog{name=barCatalog, location=/tmp/mylocation}", hadoopCatalog.get().toString()); + assertThat(hadoopCatalog).isPresent(); + assertThat(hadoopCatalog.get()).isInstanceOf(HadoopCatalog.class); + assertThat(hadoopCatalog.get().toString()) + .isEqualTo("HadoopCatalog{name=barCatalog, location=/tmp/mylocation}"); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertFalse(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isFalse(); } @Test @@ -250,16 +251,16 @@ public void testLoadCatalogCustom() { catalogName, CatalogProperties.WAREHOUSE_LOCATION), "/tmp/mylocation"); Optional customHadoopCatalog = Catalogs.loadCatalog(conf, catalogName); - Assert.assertTrue(customHadoopCatalog.isPresent()); - Assertions.assertThat(customHadoopCatalog.get()).isInstanceOf(CustomHadoopCatalog.class); + assertThat(customHadoopCatalog).isPresent(); + assertThat(customHadoopCatalog.get()).isInstanceOf(CustomHadoopCatalog.class); Properties properties = new Properties(); properties.put(InputFormatConfig.CATALOG_NAME, catalogName); - Assert.assertFalse(Catalogs.hiveCatalog(conf, properties)); + assertThat(Catalogs.hiveCatalog(conf, properties)).isFalse(); } @Test public void testLoadCatalogLocation() { - Assert.assertFalse(Catalogs.loadCatalog(conf, Catalogs.ICEBERG_HADOOP_TABLE_NAME).isPresent()); + assertThat(Catalogs.loadCatalog(conf, Catalogs.ICEBERG_HADOOP_TABLE_NAME)).isNotPresent(); } @Test @@ -269,7 +270,7 @@ public void testLoadCatalogUnknown() { InputFormatConfig.catalogPropertyConfigKey(catalogName, CatalogUtil.ICEBERG_CATALOG_TYPE), "fooType"); - Assertions.assertThatThrownBy(() -> Catalogs.loadCatalog(conf, catalogName)) + assertThatThrownBy(() -> Catalogs.loadCatalog(conf, catalogName)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Unknown catalog type: fooType"); } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java index de5189f7c3ad..43a61d83e434 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -63,7 +64,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; -import org.junit.Assert; public class HiveIcebergTestUtils { // TODO: Can this be a constant all around the Iceberg tests? @@ -218,13 +218,10 @@ public static void assertEquals(Record expected, Record actual) { for (int i = 0; i < expected.size(); ++i) { if (expected.get(i) instanceof OffsetDateTime) { // For OffsetDateTime we just compare the actual instant - Assert.assertEquals( - ((OffsetDateTime) expected.get(i)).toInstant(), - ((OffsetDateTime) actual.get(i)).toInstant()); - } else if (expected.get(i) instanceof byte[]) { - Assert.assertArrayEquals((byte[]) expected.get(i), (byte[]) actual.get(i)); + assertThat(((OffsetDateTime) actual.get(i)).toInstant()) + .isEqualTo(((OffsetDateTime) expected.get(i)).toInstant()); } else { - Assert.assertEquals(expected.get(i), actual.get(i)); + assertThat(actual.get(i)).isEqualTo(expected.get(i)); } } } @@ -265,7 +262,7 @@ public static void validateData(List expected, List actual, int sortedExpected.sort(Comparator.comparingLong(record -> (Long) record.get(sortBy))); sortedActual.sort(Comparator.comparingLong(record -> (Long) record.get(sortBy))); - Assert.assertEquals(sortedExpected.size(), sortedActual.size()); + assertThat(sortedActual).hasSameSizeAs(sortedExpected); for (int i = 0; i < sortedExpected.size(); ++i) { assertEquals(sortedExpected.get(i), sortedActual.get(i)); } @@ -288,9 +285,9 @@ public static void validateFiles(Table table, Configuration conf, JobID jobId, i .filter(path -> !path.getFileName().toString().startsWith(".")) .collect(Collectors.toList()); - Assert.assertEquals(dataFileNum, dataFiles.size()); - Assert.assertFalse( - new File(HiveIcebergOutputCommitter.generateJobLocation(table.location(), conf, jobId)) - .exists()); + assertThat(dataFiles).hasSize(dataFileNum); + assertThat( + new File(HiveIcebergOutputCommitter.generateJobLocation(table.location(), conf, jobId))) + .doesNotExist(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java index 4a5d819279f2..8f58a36d6265 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestDeserializer.java @@ -19,6 +19,8 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.Collections; @@ -35,9 +37,7 @@ import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.mr.hive.serde.objectinspector.IcebergObjectInspector; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDeserializer { private static final Schema CUSTOMER_SCHEMA = @@ -74,7 +74,7 @@ public void testSchemaDeserialize() { Record actual = deserializer.deserialize(new Object[] {new LongWritable(1L), new Text("Bob")}); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test @@ -92,7 +92,7 @@ public void testStructDeserialize() { Record actual = deserializer.deserialize(new Object[] {new LongWritable(1L), new Text("Bob")}); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test @@ -127,7 +127,7 @@ public void testMapDeserialize() { Object[] data = new Object[] {map}; Record actual = deserializer.deserialize(data); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test @@ -155,13 +155,14 @@ public void testListDeserialize() { Object[] data = new Object[] {new Object[] {new LongWritable(1L)}}; Record actual = deserializer.deserialize(data); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } @Test public void testDeserializeEverySupportedType() { - Assume.assumeFalse( - "No test yet for Hive3 (Date/Timestamp creation)", HiveVersion.min(HiveVersion.HIVE_3)); + assumeThat(HiveVersion.min(HiveVersion.HIVE_3)) + .as("No test yet for Hive3 (Date/Timestamp creation)") + .isFalse(); Deserializer deserializer = new Deserializer.Builder() @@ -196,9 +197,9 @@ public void testNullDeserialize() { Record actual = deserializer.deserialize(nulls); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); // Check null record as well - Assert.assertNull(deserializer.deserialize(null)); + assertThat(deserializer.deserialize(null)).isNull(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java index 05fe0fdea5ae..121e2c8b6d8a 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.mr.hive; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import java.math.BigDecimal; import java.sql.Date; @@ -40,7 +40,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestHiveIcebergFilterFactory { @@ -82,10 +82,10 @@ public void testNotEqualsOperand() { UnboundPredicate childExpressionActual = (UnboundPredicate) actual.child(); UnboundPredicate childExpressionExpected = Expressions.equal("salary", 3000L); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.child().op(), expected.child().op()); - assertEquals(childExpressionActual.ref().name(), childExpressionExpected.ref().name()); - assertEquals(childExpressionActual.literal(), childExpressionExpected.literal()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.child().op()).isEqualTo(actual.child().op()); + assertThat(childExpressionExpected.ref().name()).isEqualTo(childExpressionActual.ref().name()); + assertThat(childExpressionExpected.literal()).isEqualTo(childExpressionActual.literal()); } @Test @@ -98,9 +98,9 @@ public void testLessThanOperand() { UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.literal(), expected.literal()); - assertEquals(actual.ref().name(), expected.ref().name()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.literal()).isEqualTo(actual.literal()); + assertThat(expected.ref().name()).isEqualTo(actual.ref().name()); } @Test @@ -126,9 +126,9 @@ public void testInOperand() { UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.literals(), expected.literals()); - assertEquals(actual.ref().name(), expected.ref().name()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.literals()).isEqualTo(actual.literals()); + assertThat(expected.ref().name()).isEqualTo(actual.ref().name()); } @Test @@ -144,9 +144,9 @@ public void testBetweenOperand() { Expressions.lessThanOrEqual("salary", 3000L)); And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.left().op(), expected.left().op()); - assertEquals(actual.right().op(), expected.right().op()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.left().op()).isEqualTo(actual.left().op()); + assertThat(expected.right().op()).isEqualTo(actual.right().op()); } @Test @@ -173,8 +173,8 @@ public void testIsNullOperand() { UnboundPredicate actual = (UnboundPredicate) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.ref().name(), expected.ref().name()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.ref().name()).isEqualTo(actual.ref().name()); } @Test @@ -193,9 +193,9 @@ public void testAndOperand() { Expressions.and(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L)); And actual = (And) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.left().op(), expected.left().op()); - assertEquals(actual.right().op(), expected.right().op()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.left().op()).isEqualTo(actual.left().op()); + assertThat(expected.right().op()).isEqualTo(actual.right().op()); } @Test @@ -213,9 +213,9 @@ public void testOrOperand() { (Or) Expressions.or(Expressions.equal("salary", 3000L), Expressions.equal("salary", 4000L)); Or actual = (Or) HiveIcebergFilterFactory.generateFilterExpression(arg); - assertEquals(actual.op(), expected.op()); - assertEquals(actual.left().op(), expected.left().op()); - assertEquals(actual.right().op(), expected.right().op()); + assertThat(expected.op()).isEqualTo(actual.op()); + assertThat(expected.left().op()).isEqualTo(actual.left().op()); + assertThat(expected.right().op()).isEqualTo(actual.right().op()); } @Test @@ -308,9 +308,9 @@ public void testDecimalType() { } private void assertPredicatesMatch(UnboundPredicate expected, UnboundPredicate actual) { - assertEquals(expected.op(), actual.op()); - assertEquals(expected.literal(), actual.literal()); - assertEquals(expected.ref().name(), actual.ref().name()); + assertThat(actual.op()).isEqualTo(expected.op()); + assertThat(actual.literal()).isEqualTo(expected.literal()); + assertThat(actual.ref().name()).isEqualTo(expected.ref().name()); } private static class MockSearchArgument implements SearchArgument { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java index 0d9b5a0c1d5e..8b8e209144fa 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java @@ -20,8 +20,10 @@ import static org.apache.iceberg.mr.hive.HiveIcebergRecordWriter.getWriters; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import java.util.Map; @@ -55,10 +57,8 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializationUtil; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -80,7 +80,7 @@ public class TestHiveIcebergOutputCommitter { private static final PartitionSpec PARTITIONED_SPEC = PartitionSpec.builderFor(CUSTOMER_SCHEMA).bucket("customer_id", 3).build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testNeedsTaskCommit() { @@ -91,24 +91,25 @@ public void testNeedsTaskCommit() { mapOnlyJobConf.setNumReduceTasks(0); // Map only job should commit map tasks - Assert.assertTrue( - committer.needsTaskCommit(new TaskAttemptContextImpl(mapOnlyJobConf, MAP_TASK_ID))); + assertThat(committer.needsTaskCommit(new TaskAttemptContextImpl(mapOnlyJobConf, MAP_TASK_ID))) + .isTrue(); JobConf mapReduceJobConf = new JobConf(); mapReduceJobConf.setNumMapTasks(10); mapReduceJobConf.setNumReduceTasks(10); // MapReduce job should not commit map tasks, but should commit reduce tasks - Assert.assertFalse( - committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, MAP_TASK_ID))); - Assert.assertTrue( - committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, REDUCE_TASK_ID))); + assertThat(committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, MAP_TASK_ID))) + .isFalse(); + assertThat( + committer.needsTaskCommit(new TaskAttemptContextImpl(mapReduceJobConf, REDUCE_TASK_ID))) + .isTrue(); } @Test public void testSuccessfulUnpartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); List expected = writeRecords(table.name(), 1, 0, true, false, conf); @@ -121,7 +122,7 @@ public void testSuccessfulUnpartitionedWrite() throws IOException { @Test public void testSuccessfulPartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), true); + Table table = table(temp.toFile().getPath(), true); JobConf conf = jobConf(table, 1); List expected = writeRecords(table.name(), 1, 0, true, false, conf); committer.commitJob(new JobContextImpl(conf, JOB_ID)); @@ -133,7 +134,7 @@ public void testSuccessfulPartitionedWrite() throws IOException { @Test public void testSuccessfulMultipleTasksUnpartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 2); List expected = writeRecords(table.name(), 2, 0, true, false, conf); committer.commitJob(new JobContextImpl(conf, JOB_ID)); @@ -145,7 +146,7 @@ public void testSuccessfulMultipleTasksUnpartitionedWrite() throws IOException { @Test public void testSuccessfulMultipleTasksPartitionedWrite() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), true); + Table table = table(temp.toFile().getPath(), true); JobConf conf = jobConf(table, 2); List expected = writeRecords(table.name(), 2, 0, true, false, conf); committer.commitJob(new JobContextImpl(conf, JOB_ID)); @@ -157,7 +158,7 @@ public void testSuccessfulMultipleTasksPartitionedWrite() throws IOException { @Test public void testRetryTask() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 2); // Write records and abort the tasks @@ -181,7 +182,7 @@ public void testRetryTask() throws IOException { @Test public void testAbortJob() throws IOException { HiveIcebergOutputCommitter committer = new HiveIcebergOutputCommitter(); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); writeRecords(table.name(), 1, 0, true, false, conf); committer.abortJob(new JobContextImpl(conf, JOB_ID), JobStatus.State.FAILED); @@ -201,7 +202,7 @@ public void writerIsClosedAfterTaskCommitFailure() throws IOException { .when(failingCommitter) .commitTask(argumentCaptor.capture()); - Table table = table(temp.getRoot().getPath(), false); + Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); Assertions.assertThatThrownBy( @@ -209,14 +210,14 @@ public void writerIsClosedAfterTaskCommitFailure() throws IOException { .isInstanceOf(RuntimeException.class) .hasMessage(exceptionMessage); - Assert.assertEquals(1, argumentCaptor.getAllValues().size()); + assertThat(argumentCaptor.getAllValues()).hasSize(1); TaskAttemptID capturedId = TezUtil.taskAttemptWrapper(argumentCaptor.getValue().getTaskAttemptID()); // writer is still in the map after commitTask failure - Assert.assertNotNull(getWriters(capturedId)); + assertThat(getWriters(capturedId)).isNotNull(); failingCommitter.abortTask(new TaskAttemptContextImpl(conf, capturedId)); // abortTask succeeds and removes writer - Assert.assertNull(getWriters(capturedId)); + assertThat(getWriters(capturedId)).isNull(); } private Table table(String location, boolean partitioned) { 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 889c441c28fc..919230a9fbd9 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 @@ -19,9 +19,11 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Properties; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.serde2.SerDeException; @@ -34,22 +36,20 @@ import org.apache.iceberg.mr.hive.serde.objectinspector.IcebergObjectInspector; import org.apache.iceberg.mr.mapred.Container; 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 TestHiveIcebergSerDe { private static final Schema schema = new Schema(required(1, "string_field", Types.StringType.get())); - @Rule public TemporaryFolder tmp = new TemporaryFolder(); + @TempDir private Path tmp; @Test public void testInitialize() throws IOException, SerDeException { - File location = tmp.newFolder(); - Assert.assertTrue(location.delete()); + File location = tmp.toFile(); + assertThat(location.delete()).isTrue(); Configuration conf = new Configuration(); @@ -63,7 +63,7 @@ public void testInitialize() throws IOException, SerDeException { HiveIcebergSerDe serDe = new HiveIcebergSerDe(); serDe.initialize(conf, properties); - Assert.assertEquals(IcebergObjectInspector.create(schema), serDe.getObjectInspector()); + assertThat(serDe.getObjectInspector()).isEqualTo(IcebergObjectInspector.create(schema)); } @Test @@ -74,6 +74,6 @@ public void testDeserialize() { Container container = new Container<>(); container.set(record); - Assert.assertEquals(record, serDe.deserialize(container)); + assertThat(serDe.deserialize(container)).isEqualTo(record); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java index 5db84e5aa4b9..87fe4208c964 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergBinaryObjectInspector.java @@ -18,14 +18,15 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.ByteBuffer; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.BytesWritable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergBinaryObjectInspector { @@ -33,39 +34,40 @@ public class TestIcebergBinaryObjectInspector { public void testIcebergByteBufferObjectInspector() { BinaryObjectInspector oi = IcebergBinaryObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.BINARY, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.BINARY); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.binaryTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.binaryTypeInfo.getTypeName()); - Assert.assertEquals(byte[].class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(BytesWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(byte[].class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(BytesWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); byte[] bytes = new byte[] {0, 1, 2, 3}; ByteBuffer buffer = ByteBuffer.wrap(bytes); - Assert.assertArrayEquals(bytes, oi.getPrimitiveJavaObject(buffer)); - Assert.assertEquals(new BytesWritable(bytes), oi.getPrimitiveWritableObject(buffer)); + assertThat(oi.getPrimitiveJavaObject(buffer)).isEqualTo(bytes); + assertThat(oi.getPrimitiveWritableObject(buffer)).isEqualTo(new BytesWritable(bytes)); ByteBuffer slice = ByteBuffer.wrap(bytes, 1, 2).slice(); - Assert.assertArrayEquals(new byte[] {1, 2}, oi.getPrimitiveJavaObject(slice)); - Assert.assertEquals(new BytesWritable(new byte[] {1, 2}), oi.getPrimitiveWritableObject(slice)); + assertThat(oi.getPrimitiveJavaObject(slice)).isEqualTo(new byte[] {1, 2}); + assertThat(oi.getPrimitiveWritableObject(slice)) + .isEqualTo(new BytesWritable(new byte[] {1, 2})); slice.position(1); - Assert.assertArrayEquals(new byte[] {2}, oi.getPrimitiveJavaObject(slice)); - Assert.assertEquals(new BytesWritable(new byte[] {2}), oi.getPrimitiveWritableObject(slice)); + assertThat(oi.getPrimitiveJavaObject(slice)).isEqualTo(new byte[] {2}); + assertThat(oi.getPrimitiveWritableObject(slice)).isEqualTo(new BytesWritable(new byte[] {2})); byte[] copy = (byte[]) oi.copyObject(bytes); - Assert.assertArrayEquals(bytes, copy); - Assert.assertNotSame(bytes, copy); + assertThat(copy).isEqualTo(bytes); + assertThat(copy).isNotSameAs(bytes); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java index 73681fec5799..6e03fae861f9 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspector.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Date; import java.time.LocalDate; import org.apache.hadoop.hive.serde2.io.DateWritable; @@ -25,8 +27,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergDateObjectInspector { @@ -34,30 +35,31 @@ public class TestIcebergDateObjectInspector { public void testIcebergDateObjectInspector() { DateObjectInspector oi = IcebergDateObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals(PrimitiveObjectInspector.PrimitiveCategory.DATE, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.DATE); - Assert.assertEquals(TypeInfoFactory.dateTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.dateTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.dateTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.dateTypeInfo.getTypeName()); - Assert.assertEquals(Date.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(DateWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Date.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(DateWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); LocalDate local = LocalDate.of(2020, 1, 1); Date date = Date.valueOf("2020-01-01"); - Assert.assertEquals(date, oi.getPrimitiveJavaObject(local)); - Assert.assertEquals(new DateWritable(date), oi.getPrimitiveWritableObject(local)); + assertThat(oi.getPrimitiveJavaObject(local)).isEqualTo(date); + assertThat(oi.getPrimitiveWritableObject(local)).isEqualTo(new DateWritable(date)); Date copy = (Date) oi.copyObject(date); - Assert.assertEquals(date, copy); - Assert.assertNotSame(date, copy); + assertThat(copy).isEqualTo(date); + assertThat(copy).isNotSameAs(date); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java index 1c4734c77f4c..58d43e3d7047 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDecimalObjectInspector.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; @@ -26,8 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergDecimalObjectInspector { @@ -35,43 +36,44 @@ public class TestIcebergDecimalObjectInspector { public void testCache() { HiveDecimalObjectInspector oi = IcebergDecimalObjectInspector.get(38, 18); - Assert.assertSame(oi, IcebergDecimalObjectInspector.get(38, 18)); - Assert.assertNotSame(oi, IcebergDecimalObjectInspector.get(28, 18)); - Assert.assertNotSame(oi, IcebergDecimalObjectInspector.get(38, 28)); + assertThat(IcebergDecimalObjectInspector.get(38, 18)).isSameAs(oi); + assertThat(IcebergDecimalObjectInspector.get(28, 18)).isNotSameAs(oi); + assertThat(IcebergDecimalObjectInspector.get(38, 28)).isNotSameAs(oi); } @Test public void testIcebergDecimalObjectInspector() { HiveDecimalObjectInspector oi = IcebergDecimalObjectInspector.get(38, 18); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.DECIMAL, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.DECIMAL); - Assert.assertEquals(new DecimalTypeInfo(38, 18), oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.decimalTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(new DecimalTypeInfo(38, 18)); + assertThat(oi.getTypeName()) + .isEqualTo(TypeInfoFactory.decimalTypeInfo.getTypeName(), oi.getTypeName()); - Assert.assertEquals(38, oi.precision()); - Assert.assertEquals(18, oi.scale()); + assertThat(oi.precision()).isEqualTo(38); + assertThat(oi.scale()).isEqualTo(18); - Assert.assertEquals(HiveDecimal.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(HiveDecimalWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(HiveDecimal.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(HiveDecimalWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); HiveDecimal one = HiveDecimal.create(BigDecimal.ONE); - Assert.assertEquals(one, oi.getPrimitiveJavaObject(BigDecimal.ONE)); - Assert.assertEquals( - new HiveDecimalWritable(one), oi.getPrimitiveWritableObject(BigDecimal.ONE)); + assertThat(oi.getPrimitiveJavaObject(BigDecimal.ONE)).isEqualTo(one); + assertThat(oi.getPrimitiveWritableObject(BigDecimal.ONE)) + .isEqualTo(new HiveDecimalWritable(one)); HiveDecimal copy = (HiveDecimal) oi.copyObject(one); - Assert.assertEquals(one, copy); - Assert.assertNotSame(one, copy); + assertThat(copy).isEqualTo(one); + assertThat(copy).isNotSameAs(one); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java index a902d027269d..7a8450f6852f 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergFixedObjectInspector.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.BytesWritable; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergFixedObjectInspector { @@ -31,33 +32,33 @@ public class TestIcebergFixedObjectInspector { public void testIcebergFixedObjectInspector() { IcebergFixedObjectInspector oi = IcebergFixedObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.BINARY, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.BINARY); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.binaryTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.binaryTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.binaryTypeInfo.getTypeName()); - Assert.assertEquals(byte[].class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(BytesWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(byte[].class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(BytesWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); byte[] bytes = new byte[] {0, 1}; BytesWritable bytesWritable = new BytesWritable(bytes); - Assert.assertArrayEquals(bytes, oi.getPrimitiveJavaObject(bytes)); - Assert.assertEquals(bytesWritable, oi.getPrimitiveWritableObject(bytes)); - Assert.assertEquals(bytes, oi.convert(bytes)); + assertThat(oi.getPrimitiveJavaObject(bytes)).isEqualTo(bytes); + assertThat(oi.getPrimitiveWritableObject(bytes)).isEqualTo(bytesWritable); + assertThat(oi.convert(bytes)).isEqualTo(bytes); byte[] copy = (byte[]) oi.copyObject(bytes); - Assert.assertArrayEquals(bytes, copy); - Assert.assertNotSame(bytes, copy); + assertThat(copy).isEqualTo(bytes); + assertThat(copy).isNotSameAs(bytes); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java index c4d3cea4b002..c2646376890c 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergObjectInspector.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive.serde.objectinspector; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; @@ -31,8 +32,7 @@ import org.apache.iceberg.hive.HiveVersion; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergObjectInspector { @@ -74,165 +74,161 @@ public class TestIcebergObjectInspector { @Test public void testIcebergObjectInspector() { ObjectInspector oi = IcebergObjectInspector.create(schema); - Assert.assertNotNull(oi); - Assert.assertEquals(ObjectInspector.Category.STRUCT, oi.getCategory()); + assertThat(oi).isNotNull(); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.STRUCT); StructObjectInspector soi = (StructObjectInspector) oi; // binary StructField binaryField = soi.getStructFieldRef("binary_field"); - Assert.assertEquals(1, binaryField.getFieldID()); - Assert.assertEquals("binary_field", binaryField.getFieldName()); - Assert.assertEquals("binary comment", binaryField.getFieldComment()); - Assert.assertEquals(IcebergBinaryObjectInspector.get(), binaryField.getFieldObjectInspector()); + assertThat(binaryField.getFieldID()).isEqualTo(1); + assertThat(binaryField.getFieldName()).isEqualTo("binary_field"); + assertThat(binaryField.getFieldComment()).isEqualTo("binary comment"); + assertThat(binaryField.getFieldObjectInspector()).isEqualTo(IcebergBinaryObjectInspector.get()); // boolean StructField booleanField = soi.getStructFieldRef("boolean_field"); - Assert.assertEquals(2, booleanField.getFieldID()); - Assert.assertEquals("boolean_field", booleanField.getFieldName()); - Assert.assertEquals("boolean comment", booleanField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(boolean.class), booleanField.getFieldObjectInspector()); + assertThat(booleanField.getFieldID()).isEqualTo(2); + assertThat(booleanField.getFieldName()).isEqualTo("boolean_field"); + assertThat(booleanField.getFieldComment()).isEqualTo("boolean comment"); + assertThat(booleanField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(boolean.class)); // date StructField dateField = soi.getStructFieldRef("date_field"); - Assert.assertEquals(3, dateField.getFieldID()); - Assert.assertEquals("date_field", dateField.getFieldName()); - Assert.assertEquals("date comment", dateField.getFieldComment()); + assertThat(dateField.getFieldID()).isEqualTo(3); + assertThat(dateField.getFieldName()).isEqualTo("date_field"); + assertThat(dateField.getFieldComment()).isEqualTo("date comment"); if (HiveVersion.min(HiveVersion.HIVE_3)) { - Assert.assertEquals( - "org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspectorHive3", - dateField.getFieldObjectInspector().getClass().getName()); + assertThat(dateField.getFieldObjectInspector().getClass().getName()) + .isEqualTo( + "org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspectorHive3"); } else { - Assert.assertEquals( - "org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspector", - dateField.getFieldObjectInspector().getClass().getName()); + assertThat(dateField.getFieldObjectInspector().getClass().getName()) + .isEqualTo("org.apache.iceberg.mr.hive.serde.objectinspector.IcebergDateObjectInspector"); } // decimal StructField decimalField = soi.getStructFieldRef("decimal_field"); - Assert.assertEquals(4, decimalField.getFieldID()); - Assert.assertEquals("decimal_field", decimalField.getFieldName()); - Assert.assertEquals("decimal comment", decimalField.getFieldComment()); - Assert.assertEquals( - IcebergDecimalObjectInspector.get(38, 18), decimalField.getFieldObjectInspector()); + assertThat(decimalField.getFieldID()).isEqualTo(4); + assertThat(decimalField.getFieldName()).isEqualTo("decimal_field"); + assertThat(decimalField.getFieldComment()).isEqualTo("decimal comment"); + assertThat(decimalField.getFieldObjectInspector()) + .isEqualTo(IcebergDecimalObjectInspector.get(38, 18)); // double StructField doubleField = soi.getStructFieldRef("double_field"); - Assert.assertEquals(5, doubleField.getFieldID()); - Assert.assertEquals("double_field", doubleField.getFieldName()); - Assert.assertEquals("double comment", doubleField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(double.class), doubleField.getFieldObjectInspector()); + assertThat(doubleField.getFieldID()).isEqualTo(5); + assertThat(doubleField.getFieldName()).isEqualTo("double_field"); + assertThat(doubleField.getFieldComment()).isEqualTo("double comment"); + assertThat(doubleField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(double.class)); // fixed StructField fixedField = soi.getStructFieldRef("fixed_field"); - Assert.assertEquals(6, fixedField.getFieldID()); - Assert.assertEquals("fixed_field", fixedField.getFieldName()); - Assert.assertEquals("fixed comment", fixedField.getFieldComment()); - Assert.assertEquals(IcebergFixedObjectInspector.get(), fixedField.getFieldObjectInspector()); + assertThat(fixedField.getFieldID()).isEqualTo(6); + assertThat(fixedField.getFieldName()).isEqualTo("fixed_field"); + assertThat(fixedField.getFieldComment()).isEqualTo("fixed comment"); + assertThat(fixedField.getFieldObjectInspector()).isEqualTo(IcebergFixedObjectInspector.get()); // float StructField floatField = soi.getStructFieldRef("float_field"); - Assert.assertEquals(7, floatField.getFieldID()); - Assert.assertEquals("float_field", floatField.getFieldName()); - Assert.assertEquals("float comment", floatField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(float.class), floatField.getFieldObjectInspector()); + assertThat(floatField.getFieldID()).isEqualTo(7); + assertThat(floatField.getFieldName()).isEqualTo("float_field"); + assertThat(floatField.getFieldComment()).isEqualTo("float comment"); + assertThat(floatField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(float.class)); // integer StructField integerField = soi.getStructFieldRef("integer_field"); - Assert.assertEquals(8, integerField.getFieldID()); - Assert.assertEquals("integer_field", integerField.getFieldName()); - Assert.assertEquals("integer comment", integerField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(int.class), integerField.getFieldObjectInspector()); + assertThat(integerField.getFieldID()).isEqualTo(8); + assertThat(integerField.getFieldName()).isEqualTo("integer_field"); + assertThat(integerField.getFieldComment()).isEqualTo("integer comment"); + assertThat(integerField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(int.class)); // long StructField longField = soi.getStructFieldRef("long_field"); - Assert.assertEquals(9, longField.getFieldID()); - Assert.assertEquals("long_field", longField.getFieldName()); - Assert.assertEquals("long comment", longField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(long.class), longField.getFieldObjectInspector()); + assertThat(longField.getFieldID()).isEqualTo(9); + assertThat(longField.getFieldName()).isEqualTo("long_field"); + assertThat(longField.getFieldComment()).isEqualTo("long comment"); + assertThat(longField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(long.class)); // string StructField stringField = soi.getStructFieldRef("string_field"); - Assert.assertEquals(10, stringField.getFieldID()); - Assert.assertEquals("string_field", stringField.getFieldName()); - Assert.assertEquals("string comment", stringField.getFieldComment()); - Assert.assertEquals( - getPrimitiveObjectInspector(String.class), stringField.getFieldObjectInspector()); + assertThat(stringField.getFieldID()).isEqualTo(10); + assertThat(stringField.getFieldName()).isEqualTo("string_field"); + assertThat(stringField.getFieldComment()).isEqualTo("string comment"); + assertThat(stringField.getFieldObjectInspector()) + .isEqualTo(getPrimitiveObjectInspector(String.class)); // timestamp without tz StructField timestampField = soi.getStructFieldRef("timestamp_field"); - Assert.assertEquals(11, timestampField.getFieldID()); - Assert.assertEquals("timestamp_field", timestampField.getFieldName()); - Assert.assertEquals("timestamp comment", timestampField.getFieldComment()); + assertThat(timestampField.getFieldID()).isEqualTo(11); + assertThat(timestampField.getFieldName()).isEqualTo("timestamp_field"); + assertThat(timestampField.getFieldComment()).isEqualTo("timestamp comment"); if (HiveVersion.min(HiveVersion.HIVE_3)) { - Assert.assertEquals( - "IcebergTimestampObjectInspectorHive3", - timestampField.getFieldObjectInspector().getClass().getSimpleName()); + assertThat(timestampField.getFieldObjectInspector().getClass().getSimpleName()) + .isEqualTo("IcebergTimestampObjectInspectorHive3"); } else { - Assert.assertEquals( - IcebergTimestampObjectInspector.get(), timestampField.getFieldObjectInspector()); + assertThat(timestampField.getFieldObjectInspector()) + .isEqualTo(IcebergTimestampObjectInspector.get()); } // timestamp with tz StructField timestampTzField = soi.getStructFieldRef("timestamptz_field"); - Assert.assertEquals(12, timestampTzField.getFieldID()); - Assert.assertEquals("timestamptz_field", timestampTzField.getFieldName()); - Assert.assertEquals("timestamptz comment", timestampTzField.getFieldComment()); + assertThat(timestampTzField.getFieldID()).isEqualTo(12); + assertThat(timestampTzField.getFieldName()).isEqualTo("timestamptz_field"); + assertThat(timestampTzField.getFieldComment()).isEqualTo("timestamptz comment"); if (HiveVersion.min(HiveVersion.HIVE_3)) { - Assert.assertEquals( - "IcebergTimestampWithZoneObjectInspectorHive3", - timestampTzField.getFieldObjectInspector().getClass().getSimpleName()); + assertThat(timestampTzField.getFieldObjectInspector().getClass().getSimpleName()) + .isEqualTo("IcebergTimestampWithZoneObjectInspectorHive3"); } else { - Assert.assertEquals( - IcebergTimestampWithZoneObjectInspector.get(), - timestampTzField.getFieldObjectInspector()); + assertThat(timestampTzField.getFieldObjectInspector()) + .isEqualTo(IcebergTimestampWithZoneObjectInspector.get()); } // UUID StructField uuidField = soi.getStructFieldRef("uuid_field"); - Assert.assertEquals(13, uuidField.getFieldID()); - Assert.assertEquals("uuid_field", uuidField.getFieldName()); - Assert.assertEquals("uuid comment", uuidField.getFieldComment()); - Assert.assertEquals(IcebergUUIDObjectInspector.get(), uuidField.getFieldObjectInspector()); + assertThat(uuidField.getFieldID()).isEqualTo(13); + assertThat(uuidField.getFieldName()).isEqualTo("uuid_field"); + assertThat(uuidField.getFieldComment()).isEqualTo("uuid comment"); + assertThat(uuidField.getFieldObjectInspector()).isEqualTo(IcebergUUIDObjectInspector.get()); // list StructField listField = soi.getStructFieldRef("list_field"); - Assert.assertEquals(14, listField.getFieldID()); - Assert.assertEquals("list_field", listField.getFieldName()); - Assert.assertEquals("list comment", listField.getFieldComment()); - Assert.assertEquals(getListObjectInspector(String.class), listField.getFieldObjectInspector()); + assertThat(listField.getFieldID()).isEqualTo(14); + assertThat(listField.getFieldName()).isEqualTo("list_field"); + assertThat(listField.getFieldComment()).isEqualTo("list comment"); + assertThat(listField.getFieldObjectInspector()).isEqualTo(getListObjectInspector(String.class)); // map StructField mapField = soi.getStructFieldRef("map_field"); - Assert.assertEquals(16, mapField.getFieldID()); - Assert.assertEquals("map_field", mapField.getFieldName()); - Assert.assertEquals("map comment", mapField.getFieldComment()); - Assert.assertEquals( - getMapObjectInspector(String.class, int.class), mapField.getFieldObjectInspector()); + assertThat(mapField.getFieldID()).isEqualTo(16); + assertThat(mapField.getFieldName()).isEqualTo("map_field"); + assertThat(mapField.getFieldComment()).isEqualTo("map comment"); + assertThat(mapField.getFieldObjectInspector()) + .isEqualTo(getMapObjectInspector(String.class, int.class)); // struct StructField structField = soi.getStructFieldRef("struct_field"); - Assert.assertEquals(19, structField.getFieldID()); - Assert.assertEquals("struct_field", structField.getFieldName()); - Assert.assertEquals("struct comment", structField.getFieldComment()); + assertThat(structField.getFieldID()).isEqualTo(19); + assertThat(structField.getFieldName()).isEqualTo("struct_field"); + assertThat(structField.getFieldComment()).isEqualTo("struct comment"); ObjectInspector expectedObjectInspector = new IcebergRecordObjectInspector( (Types.StructType) schema.findType(19), ImmutableList.of(getPrimitiveObjectInspector(String.class))); - Assert.assertEquals(expectedObjectInspector, structField.getFieldObjectInspector()); + assertThat(structField.getFieldObjectInspector()).isEqualTo(expectedObjectInspector); // time StructField timeField = soi.getStructFieldRef("time_field"); - Assert.assertEquals(21, timeField.getFieldID()); - Assert.assertEquals("time_field", timeField.getFieldName()); - Assert.assertEquals("time comment", timeField.getFieldComment()); - Assert.assertEquals(IcebergTimeObjectInspector.get(), timeField.getFieldObjectInspector()); + assertThat(timeField.getFieldID()).isEqualTo(21); + assertThat(timeField.getFieldName()).isEqualTo("time_field"); + assertThat(timeField.getFieldComment()).isEqualTo("time comment"); + assertThat(timeField.getFieldObjectInspector()).isEqualTo(IcebergTimeObjectInspector.get()); } private static ObjectInspector getPrimitiveObjectInspector(Class clazz) { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java index 4ed358c116fb..d5824f8bd7d8 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergRecordObjectInspector.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive.serde.objectinspector; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; @@ -27,8 +28,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergRecordObjectInspector { @@ -47,22 +47,22 @@ public void testIcebergRecordObjectInspector() { Record innerRecord = record.get(1, Record.class); StructObjectInspector soi = (StructObjectInspector) IcebergObjectInspector.create(schema); - Assert.assertEquals( - ImmutableList.of(record.get(0), record.get(1)), soi.getStructFieldsDataAsList(record)); + assertThat(soi.getStructFieldsDataAsList(record)) + .isEqualTo(ImmutableList.of(record.get(0), record.get(1))); StructField integerField = soi.getStructFieldRef("integer_field"); - Assert.assertEquals(record.get(0), soi.getStructFieldData(record, integerField)); + assertThat(soi.getStructFieldData(record, integerField)).isEqualTo(record.get(0)); StructField structField = soi.getStructFieldRef("struct_field"); Object innerData = soi.getStructFieldData(record, structField); - Assert.assertEquals(innerRecord, innerData); + assertThat(innerData).isEqualTo(innerRecord); StructObjectInspector innerSoi = (StructObjectInspector) structField.getFieldObjectInspector(); StructField stringField = innerSoi.getStructFieldRef("string_field"); - Assert.assertEquals( - ImmutableList.of(innerRecord.get(0)), innerSoi.getStructFieldsDataAsList(innerRecord)); - Assert.assertEquals(innerRecord.get(0), innerSoi.getStructFieldData(innerData, stringField)); + assertThat(innerSoi.getStructFieldsDataAsList(innerRecord)) + .isEqualTo(ImmutableList.of(innerRecord.get(0))); + assertThat(innerSoi.getStructFieldData(innerData, stringField)).isEqualTo(innerRecord.get(0)); } @Test @@ -76,8 +76,8 @@ public void testIcebergRecordObjectInspectorWithRowNull() { Types.StructType.of( Types.NestedField.required(3, "string_field", Types.StringType.get())))); StructObjectInspector soi = (StructObjectInspector) IcebergObjectInspector.create(schema); - Assert.assertNull(soi.getStructFieldsDataAsList(null)); + assertThat(soi.getStructFieldsDataAsList(null)).isNull(); StructField integerField = soi.getStructFieldRef("integer_field"); - Assert.assertNull(soi.getStructFieldData(null, integerField)); + assertThat(soi.getStructFieldData(null, integerField)).isNull(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java index 04c3c710f27d..5af9ba341ebd 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimeObjectInspector.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.LocalTime; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.Text; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimeObjectInspector { @@ -33,34 +34,34 @@ public void testIcebergTimeObjectInspector() { IcebergTimeObjectInspector oi = IcebergTimeObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.STRING, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.STRING); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.stringTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.stringTypeInfo.getTypeName()); - Assert.assertEquals(String.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(Text.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(String.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(Text.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalTime localTime = LocalTime.now(); String time = localTime.toString(); Text text = new Text(time); - Assert.assertEquals(time, oi.getPrimitiveJavaObject(text)); - Assert.assertEquals(text, oi.getPrimitiveWritableObject(time)); - Assert.assertEquals(localTime, oi.convert(time)); + assertThat(oi.getPrimitiveJavaObject(text)).isEqualTo(time); + assertThat(oi.getPrimitiveWritableObject(time)).isEqualTo(text); + assertThat(oi.convert(time)).isEqualTo(localTime); Text copy = (Text) oi.copyObject(text); - Assert.assertEquals(text, copy); - Assert.assertNotSame(text, copy); + assertThat(copy).isEqualTo(text); + assertThat(copy).isNotSameAs(text); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java index 9205d7c0d7f8..ea40cc20420a 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspector.java @@ -18,14 +18,15 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Timestamp; import java.time.LocalDateTime; import org.apache.hadoop.hive.serde2.io.TimestampWritable; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimestampObjectInspector { @@ -33,34 +34,34 @@ public class TestIcebergTimestampObjectInspector { public void testIcebergTimestampObjectInspector() { IcebergTimestampObjectInspector oi = IcebergTimestampObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.timestampTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.timestampTypeInfo.getTypeName()); - Assert.assertEquals(Timestamp.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(TimestampWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Timestamp.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(TimestampWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalDateTime local = LocalDateTime.of(2020, 1, 1, 12, 55, 30, 5560000); Timestamp ts = Timestamp.valueOf(local); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(local)); - Assert.assertEquals(new TimestampWritable(ts), oi.getPrimitiveWritableObject(local)); + assertThat(oi.getPrimitiveJavaObject(local)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(local)).isEqualTo(new TimestampWritable(ts)); Timestamp copy = (Timestamp) oi.copyObject(ts); - Assert.assertEquals(ts, copy); - Assert.assertNotSame(ts, copy); + assertThat(copy).isEqualTo(ts); + assertThat(copy).isNotSameAs(ts); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); - Assert.assertEquals(local, oi.convert(ts)); + assertThat(oi.convert(ts)).isEqualTo(local); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java index 20caec44f7bd..1b16e6e02c0e 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspector.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.OffsetDateTime; @@ -26,8 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimestampWithZoneObjectInspector { @@ -35,41 +36,40 @@ public class TestIcebergTimestampWithZoneObjectInspector { public void testIcebergTimestampObjectInspectorWithUTCAdjustment() { IcebergTimestampWithZoneObjectInspector oi = IcebergTimestampWithZoneObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.timestampTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.timestampTypeInfo.getTypeName()); - Assert.assertEquals(Timestamp.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(TimestampWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Timestamp.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(TimestampWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalDateTime local = LocalDateTime.of(2020, 1, 1, 16, 45, 33, 456000); OffsetDateTime offsetDateTime = OffsetDateTime.of(local, ZoneOffset.ofHours(-5)); Timestamp ts = Timestamp.from(offsetDateTime.toInstant()); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(offsetDateTime)); - Assert.assertEquals(new TimestampWritable(ts), oi.getPrimitiveWritableObject(offsetDateTime)); + assertThat(oi.getPrimitiveJavaObject(offsetDateTime)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(offsetDateTime)).isEqualTo(new TimestampWritable(ts)); Timestamp copy = (Timestamp) oi.copyObject(ts); - Assert.assertEquals(ts, copy); - Assert.assertNotSame(ts, copy); + assertThat(copy).isEqualTo(ts); + assertThat(copy).isNotSameAs(ts); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); - Assert.assertEquals( - OffsetDateTime.ofInstant(local.toInstant(ZoneOffset.ofHours(-5)), ZoneOffset.UTC), - oi.convert(ts)); + assertThat(oi.convert(ts)) + .isEqualTo( + OffsetDateTime.ofInstant(local.toInstant(ZoneOffset.ofHours(-5)), ZoneOffset.UTC)); - Assert.assertEquals( - offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC), - oi.convert(Timestamp.from(offsetDateTime.toInstant()))); + assertThat(offsetDateTime.withOffsetSameInstant(ZoneOffset.UTC)) + .isEqualTo(oi.convert(Timestamp.from(offsetDateTime.toInstant()))); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java index 303cabc1cc15..abc0c01ed8d1 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergUUIDObjectInspector.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.UUID; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.io.Text; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergUUIDObjectInspector { @@ -32,34 +33,34 @@ public class TestIcebergUUIDObjectInspector { public void testIcebergUUIDObjectInspector() { IcebergUUIDObjectInspector oi = IcebergUUIDObjectInspector.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.STRING, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.STRING); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.stringTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.stringTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.stringTypeInfo.getTypeName()); - Assert.assertEquals(String.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(Text.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(String.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(Text.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); UUID uuid = UUID.randomUUID(); String uuidStr = uuid.toString(); Text text = new Text(uuidStr); - Assert.assertEquals(uuidStr, oi.getPrimitiveJavaObject(text)); - Assert.assertEquals(text, oi.getPrimitiveWritableObject(uuidStr)); - Assert.assertEquals(uuid, oi.convert(uuidStr)); + assertThat(oi.getPrimitiveJavaObject(text)).isEqualTo(uuidStr); + assertThat(oi.getPrimitiveWritableObject(uuidStr)).isEqualTo(text); + assertThat(oi.convert(uuidStr)).isEqualTo(uuid); Text copy = (Text) oi.copyObject(text); - Assert.assertEquals(text, copy); - Assert.assertNotSame(text, copy); + assertThat(copy).isEqualTo(text); + assertThat(copy).isNotSameAs(text); - Assert.assertFalse(oi.preferWritable()); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 26b3701816f5..4cbbe4a562c1 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -19,12 +19,17 @@ package org.apache.iceberg.nessie; import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; +import javax.annotation.Nullable; import org.apache.iceberg.Snapshot; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.Namespace; @@ -36,24 +41,25 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.base.Suppliers; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Tasks; import org.projectnessie.client.NessieConfigConstants; import org.projectnessie.client.api.CommitMultipleOperationsBuilder; +import org.projectnessie.client.api.GetContentBuilder; import org.projectnessie.client.api.NessieApiV1; import org.projectnessie.client.api.OnReferenceBuilder; import org.projectnessie.client.http.HttpClientException; import org.projectnessie.error.BaseNessieClientServerException; import org.projectnessie.error.NessieConflictException; -import org.projectnessie.error.NessieNamespaceAlreadyExistsException; -import org.projectnessie.error.NessieNamespaceNotEmptyException; -import org.projectnessie.error.NessieNamespaceNotFoundException; +import org.projectnessie.error.NessieContentNotFoundException; import org.projectnessie.error.NessieNotFoundException; +import org.projectnessie.error.NessieReferenceConflictException; import org.projectnessie.error.NessieReferenceNotFoundException; import org.projectnessie.model.Branch; +import org.projectnessie.model.Conflict; import org.projectnessie.model.Content; import org.projectnessie.model.ContentKey; import org.projectnessie.model.EntriesResponse; -import org.projectnessie.model.GetNamespacesResponse; import org.projectnessie.model.IcebergTable; import org.projectnessie.model.ImmutableCommitMeta; import org.projectnessie.model.ImmutableIcebergTable; @@ -181,133 +187,214 @@ public IcebergTable table(TableIdentifier tableIdentifier) { } public void createNamespace(Namespace namespace, Map metadata) { + getRef().checkMutable(); + if (namespace.isEmpty()) { + throw new IllegalArgumentException("Creating empty namespaces is not supported"); + } + ContentKey key = ContentKey.of(namespace.levels()); + org.projectnessie.model.Namespace content = + org.projectnessie.model.Namespace.of(key.getElements(), metadata); try { - getRef().checkMutable(); - withReference( - getApi() - .createNamespace() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels())) - .properties(metadata)) - .create(); - refresh(); - } catch (NessieNamespaceAlreadyExistsException e) { - throw new AlreadyExistsException(e, "Namespace already exists: %s", namespace); + Content existing = api.getContent().reference(getReference()).key(key).get().get(key); + if (existing != null) { + throw namespaceAlreadyExists(key, existing, null); + } + try { + commitRetry("create namespace " + key, Operation.Put.of(key, content)); + } catch (NessieReferenceConflictException e) { + Optional conflict = + NessieUtil.extractSingleConflict( + e, + EnumSet.of( + Conflict.ConflictType.KEY_EXISTS, Conflict.ConflictType.NAMESPACE_ABSENT)); + if (conflict.isPresent()) { + switch (conflict.get().conflictType()) { + case KEY_EXISTS: + Content conflicting = withReference(api.getContent()).key(key).get().get(key); + throw namespaceAlreadyExists(key, conflicting, e); + case NAMESPACE_ABSENT: + throw new NoSuchNamespaceException( + e, + "Cannot create namespace '%s': parent namespace '%s' does not exist", + namespace, + conflict.get().key()); + } + } + throw new RuntimeException( + String.format("Cannot create namespace '%s': %s", namespace, e.getMessage())); + } } catch (NessieNotFoundException e) { throw new RuntimeException( String.format( - "Cannot create Namespace '%s': " + "ref '%s' is no longer valid.", + "Cannot create namespace '%s': ref '%s' is no longer valid.", namespace, getRef().getName()), e); + } catch (BaseNessieClientServerException e) { + throw new RuntimeException( + String.format("Cannot create namespace '%s': %s", namespace, e.getMessage()), e); } } public List listNamespaces(Namespace namespace) throws NoSuchNamespaceException { try { - GetNamespacesResponse response = - withReference( - getApi() - .getMultipleNamespaces() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels()))) - .get(); - return response.getNamespaces().stream() - .map(ns -> Namespace.of(ns.getElements().toArray(new String[0]))) - .filter(ns -> ns.length() == namespace.length() + 1) + String filter = "entry.contentType == 'NAMESPACE' && "; + if (namespace.isEmpty()) { + filter += "size(entry.keyElements) == 1"; + } else { + org.projectnessie.model.Namespace root = + org.projectnessie.model.Namespace.of(namespace.levels()); + filter += + String.format( + "size(entry.keyElements) == %d && entry.encodedKey.startsWith('%s.')", + root.getElementCount() + 1, root.name()); + } + List entries = + withReference(api.getEntries()).filter(filter).stream() + .map(EntriesResponse.Entry::getName) + .collect(Collectors.toList()); + if (entries.isEmpty()) { + return Collections.emptyList(); + } + GetContentBuilder getContent = withReference(api.getContent()); + entries.forEach(getContent::key); + return getContent.get().values().stream() + .map(v -> v.unwrap(org.projectnessie.model.Namespace.class)) + .filter(Optional::isPresent) + .map(Optional::get) + .map(v -> Namespace.of(v.getElements().toArray(new String[0]))) .collect(Collectors.toList()); - } catch (NessieReferenceNotFoundException e) { - throw new RuntimeException( - String.format( - "Cannot list Namespaces starting from '%s': " + "ref '%s' is no longer valid.", - namespace, getRef().getName()), - e); + } catch (NessieNotFoundException e) { + if (namespace.isEmpty()) { + throw new NoSuchNamespaceException( + e, + "Cannot list top-level namespaces: ref '%s' is no longer valid.", + getRef().getName()); + } + throw new NoSuchNamespaceException( + e, + "Cannot list child namespaces from '%s': ref '%s' is no longer valid.", + namespace, + getRef().getName()); } } public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { + getRef().checkMutable(); + ContentKey key = ContentKey.of(namespace.levels()); try { - getRef().checkMutable(); - withReference( - getApi() - .deleteNamespace() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels()))) - .delete(); - refresh(); - return true; - } catch (NessieNamespaceNotFoundException e) { - return false; + Map contentMap = + api.getContent().reference(getReference()).key(key).get(); + Content existing = contentMap.get(key); + if (existing != null && !existing.getType().equals(Content.Type.NAMESPACE)) { + throw new NoSuchNamespaceException( + "Content object with name '%s' is not a namespace.", namespace); + } + try { + commitRetry("drop namespace " + key, Operation.Delete.of(key)); + return true; + } catch (NessieReferenceConflictException e) { + Optional conflict = + NessieUtil.extractSingleConflict( + e, + EnumSet.of( + Conflict.ConflictType.KEY_DOES_NOT_EXIST, + Conflict.ConflictType.NAMESPACE_NOT_EMPTY)); + if (conflict.isPresent()) { + Conflict.ConflictType conflictType = conflict.get().conflictType(); + switch (conflictType) { + case KEY_DOES_NOT_EXIST: + return false; + case NAMESPACE_NOT_EMPTY: + throw new NamespaceNotEmptyException(e, "Namespace '%s' is not empty.", namespace); + } + } + throw new RuntimeException( + String.format("Cannot drop namespace '%s': %s", namespace, e.getMessage())); + } } catch (NessieNotFoundException e) { LOG.error( - "Cannot drop Namespace '{}': ref '{}' is no longer valid.", + "Cannot drop namespace '{}': ref '{}' is no longer valid.", namespace, getRef().getName(), e); - return false; - } catch (NessieNamespaceNotEmptyException e) { - throw new NamespaceNotEmptyException( - e, "Namespace '%s' is not empty. One or more tables exist.", namespace); + } catch (BaseNessieClientServerException e) { + throw new RuntimeException( + String.format("Cannot drop namespace '%s': %s", namespace, e.getMessage()), e); } + return false; } public Map loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException { + ContentKey key = ContentKey.of(namespace.levels()); try { - return withReference( - getApi() - .getNamespace() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels()))) - .get() + Map contentMap = withReference(api.getContent()).key(key).get(); + return unwrapNamespace(contentMap.get(key)) + .orElseThrow( + () -> new NoSuchNamespaceException("Namespace does not exist: %s", namespace)) .getProperties(); - } catch (NessieNamespaceNotFoundException e) { - throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); - } catch (NessieReferenceNotFoundException e) { + } catch (NessieNotFoundException e) { throw new RuntimeException( String.format( - "Cannot load Namespace '%s': " + "ref '%s' is no longer valid.", + "Cannot load namespace '%s': ref '%s' is no longer valid.", namespace, getRef().getName()), e); } } public boolean setProperties(Namespace namespace, Map properties) { - try { - withReference( - getApi() - .updateProperties() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels())) - .updateProperties(properties)) - .update(); - refresh(); - // always successful, otherwise an exception is thrown - return true; - } catch (NessieNamespaceNotFoundException e) { - throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); - } catch (NessieNotFoundException e) { - throw new RuntimeException( - String.format( - "Cannot update properties on Namespace '%s': ref '%s' is no longer valid.", - namespace, getRef().getName()), - e); - } + return updateProperties(namespace, props -> props.putAll(properties)); } public boolean removeProperties(Namespace namespace, Set properties) { + return updateProperties(namespace, props -> props.keySet().removeAll(properties)); + } + + private boolean updateProperties(Namespace namespace, Consumer> action) { + getRef().checkMutable(); + ContentKey key = ContentKey.of(namespace.levels()); try { - withReference( - getApi() - .updateProperties() - .namespace(org.projectnessie.model.Namespace.of(namespace.levels())) - .removeProperties(properties)) - .update(); - refresh(); + commitRetry( + "update namespace " + key, + true, + commitBuilder -> { + org.projectnessie.model.Namespace oldNamespace = + unwrapNamespace(api.getContent().reference(getReference()).key(key).get().get(key)) + .orElseThrow( + () -> new NessieContentNotFoundException(key, getReference().getName())); + Map newProperties = Maps.newHashMap(oldNamespace.getProperties()); + action.accept(newProperties); + org.projectnessie.model.Namespace updatedNamespace = + org.projectnessie.model.Namespace.builder() + .from(oldNamespace) + .properties(newProperties) + .build(); + commitBuilder.operation(Operation.Put.of(key, updatedNamespace)); + return commitBuilder; + }); // always successful, otherwise an exception is thrown return true; - } catch (NessieNamespaceNotFoundException e) { - throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); - } catch (NessieNotFoundException e) { + } catch (NessieReferenceConflictException e) { + Optional conflict = + NessieUtil.extractSingleConflict(e, EnumSet.of(Conflict.ConflictType.KEY_DOES_NOT_EXIST)); + if (conflict.isPresent() + && conflict.get().conflictType() == Conflict.ConflictType.KEY_DOES_NOT_EXIST) { + throw new NoSuchNamespaceException(e, "Namespace does not exist: %s", namespace); + } + throw new RuntimeException( + String.format( + "Cannot update properties on namespace '%s': %s", namespace, e.getMessage())); + } catch (NessieContentNotFoundException e) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } catch (NessieReferenceNotFoundException e) { throw new RuntimeException( String.format( - "Cannot remove properties from Namespace '%s': ref '%s' is no longer valid.", + "Cannot update properties on namespace '%s': ref '%s' is no longer valid.", namespace, getRef().getName()), e); + } catch (BaseNessieClientServerException e) { + throw new RuntimeException( + String.format("Cannot update namespace '%s': %s", namespace, e.getMessage()), e); } } @@ -323,28 +410,11 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { throw new AlreadyExistsException("Table already exists: %s", to.name()); } - CommitMultipleOperationsBuilder operations = - getApi() - .commitMultipleOperations() - .commitMeta( - NessieUtil.buildCommitMetadata( - String.format("Iceberg rename table from '%s' to '%s'", from, to), - catalogOptions)) - .operation(Operation.Delete.of(NessieUtil.toKey(from))) - .operation(Operation.Put.of(NessieUtil.toKey(to), existingFromTable)); - try { - Tasks.foreach(operations) - .retry(5) - .stopRetryOn(NessieNotFoundException.class) - .throwFailureWhenFinished() - .onFailure((o, exception) -> refresh()) - .run( - ops -> { - Branch branch = ops.branch((Branch) getRef().getReference()).commit(); - getRef().updateReference(branch); - }, - BaseNessieClientServerException.class); + commitRetry( + String.format("Iceberg rename table from '%s' to '%s'", from, to), + Operation.Delete.of(NessieUtil.toKey(from)), + Operation.Put.of(NessieUtil.toKey(to), existingFromTable)); } catch (NessieNotFoundException e) { // important note: the NotFoundException refers to the ref only. If a table was not found it // would imply that the @@ -355,13 +425,13 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { // and removed by another. throw new RuntimeException( String.format( - "Cannot rename table '%s' to '%s': " + "ref '%s' no longer exists.", + "Cannot rename table '%s' to '%s': ref '%s' no longer exists.", from.name(), to.name(), getRef().getName()), e); } catch (BaseNessieClientServerException e) { throw new CommitFailedException( e, - "Cannot rename table '%s' to '%s': " + "the current reference is not up to date.", + "Cannot rename table '%s' to '%s': the current reference is not up to date.", from.name(), to.name()); } catch (HttpClientException ex) { @@ -390,29 +460,12 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { LOG.info("Purging data for table {} was set to true but is ignored", identifier.toString()); } - CommitMultipleOperationsBuilder commitBuilderBase = - getApi() - .commitMultipleOperations() - .commitMeta( - NessieUtil.buildCommitMetadata( - String.format("Iceberg delete table %s", identifier), catalogOptions)) - .operation(Operation.Delete.of(NessieUtil.toKey(identifier))); - // We try to drop the table. Simple retry after ref update. - boolean threw = true; try { - Tasks.foreach(commitBuilderBase) - .retry(5) - .stopRetryOn(NessieNotFoundException.class) - .throwFailureWhenFinished() - .onFailure((o, exception) -> refresh()) - .run( - commitBuilder -> { - Branch branch = commitBuilder.branch((Branch) getRef().getReference()).commit(); - getRef().updateReference(branch); - }, - BaseNessieClientServerException.class); - threw = false; + commitRetry( + String.format("Iceberg delete table %s", identifier), + Operation.Delete.of(NessieUtil.toKey(identifier))); + return true; } catch (NessieConflictException e) { LOG.error( "Cannot drop table: failed after retry (update ref '{}' and retry)", @@ -423,7 +476,7 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { } catch (BaseNessieClientServerException e) { LOG.error("Cannot drop table: unknown error", e); } - return !threw; + return false; } /** @deprecated will be removed after 1.5.0 */ @@ -540,4 +593,64 @@ public void close() { api.close(); } } + + private void commitRetry(String message, Operation... ops) + throws BaseNessieClientServerException { + commitRetry(message, false, builder -> builder.operations(Arrays.asList(ops))); + } + + private void commitRetry(String message, boolean retryConflicts, CommitEnhancer commitEnhancer) + throws BaseNessieClientServerException { + // Retry all errors except for NessieNotFoundException and also NessieConflictException, unless + // retryConflicts is set to true. + Predicate shouldRetry = + e -> + !(e instanceof NessieNotFoundException) + && (!(e instanceof NessieConflictException) || retryConflicts); + Tasks.range(1) + .retry(5) + .shouldRetryTest(shouldRetry) + .throwFailureWhenFinished() + .onFailure((o, exception) -> refresh()) + .run( + i -> { + try { + Branch branch = + commitEnhancer + .enhance(api.commitMultipleOperations()) + .commitMeta(NessieUtil.buildCommitMetadata(message, catalogOptions)) + .branch((Branch) getReference()) + .commit(); + getRef().updateReference(branch); + } catch (NessieConflictException e) { + if (retryConflicts) { + refresh(); // otherwise retrying a conflict doesn't make sense + } + throw e; + } + }, + BaseNessieClientServerException.class); + } + + private static AlreadyExistsException namespaceAlreadyExists( + ContentKey key, @Nullable Content existing, @Nullable Exception ex) { + if (existing instanceof org.projectnessie.model.Namespace) { + return new AlreadyExistsException(ex, "Namespace already exists: %s", key); + } else { + return new AlreadyExistsException( + ex, "Another content object with name '%s' already exists", key); + } + } + + private static Optional unwrapNamespace(Content content) { + return content == null + ? Optional.empty() + : content.unwrap(org.projectnessie.model.Namespace.class); + } + + private interface CommitEnhancer { + + CommitMultipleOperationsBuilder enhance(CommitMultipleOperationsBuilder builder) + throws BaseNessieClientServerException; + } } diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java index 0b4c293f8627..a5d7e7b21428 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieTableOperations.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.nessie; -import java.util.List; +import java.util.EnumSet; import java.util.Map; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.TableMetadata; @@ -34,9 +34,7 @@ import org.projectnessie.error.NessieConflictException; import org.projectnessie.error.NessieNotFoundException; import org.projectnessie.error.NessieReferenceConflictException; -import org.projectnessie.error.ReferenceConflicts; import org.projectnessie.model.Conflict; -import org.projectnessie.model.Conflict.ConflictType; import org.projectnessie.model.Content; import org.projectnessie.model.ContentKey; import org.projectnessie.model.IcebergTable; @@ -169,35 +167,33 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { } private static void maybeThrowSpecializedException(NessieReferenceConflictException ex) { - // Check if the server returned 'ReferenceConflicts' information - ReferenceConflicts referenceConflicts = ex.getErrorDetails(); - if (referenceConflicts == null) { - return; - } - - // Can only narrow down to a single exception, if there is only one conflict. - List conflicts = referenceConflicts.conflicts(); - if (conflicts.size() != 1) { - return; - } - - Conflict conflict = conflicts.get(0); - ConflictType conflictType = conflict.conflictType(); - if (conflictType != null) { - switch (conflictType) { - case NAMESPACE_ABSENT: - throw new NoSuchNamespaceException(ex, "Namespace does not exist: %s", conflict.key()); - case NAMESPACE_NOT_EMPTY: - throw new NamespaceNotEmptyException(ex, "Namespace not empty: %s", conflict.key()); - case KEY_DOES_NOT_EXIST: - throw new NoSuchTableException(ex, "Table or view does not exist: %s", conflict.key()); - case KEY_EXISTS: - throw new AlreadyExistsException(ex, "Table or view already exists: %s", conflict.key()); - default: - // Explicit fall-through - break; - } - } + NessieUtil.extractSingleConflict( + ex, + EnumSet.of( + Conflict.ConflictType.NAMESPACE_ABSENT, + Conflict.ConflictType.NAMESPACE_NOT_EMPTY, + Conflict.ConflictType.KEY_DOES_NOT_EXIST, + Conflict.ConflictType.KEY_EXISTS)) + .ifPresent( + conflict -> { + switch (conflict.conflictType()) { + case NAMESPACE_ABSENT: + throw new NoSuchNamespaceException( + ex, "Namespace does not exist: %s", conflict.key()); + case NAMESPACE_NOT_EMPTY: + throw new NamespaceNotEmptyException( + ex, "Namespace not empty: %s", conflict.key()); + case KEY_DOES_NOT_EXIST: + throw new NoSuchTableException( + ex, "Table or view does not exist: %s", conflict.key()); + case KEY_EXISTS: + throw new AlreadyExistsException( + ex, "Table or view already exists: %s", conflict.key()); + default: + // Explicit fall-through + break; + } + }); } @Override diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java index 8c69f844dbfa..3c3b0afd64d3 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieUtil.java @@ -19,9 +19,11 @@ package org.apache.iceberg.nessie; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import javax.annotation.Nullable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.SnapshotRef; @@ -32,7 +34,10 @@ 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.projectnessie.error.NessieReferenceConflictException; +import org.projectnessie.error.ReferenceConflicts; import org.projectnessie.model.CommitMeta; +import org.projectnessie.model.Conflict; import org.projectnessie.model.ContentKey; import org.projectnessie.model.IcebergTable; import org.projectnessie.model.ImmutableCommitMeta; @@ -165,4 +170,24 @@ public static TableMetadata updateTableMetadataWithNessieSpecificProperties( return builder.discardChanges().build(); } + + public static Optional extractSingleConflict( + NessieReferenceConflictException ex, Collection handledConflictTypes) { + // Check if the server returned 'ReferenceConflicts' information + ReferenceConflicts referenceConflicts = ex.getErrorDetails(); + if (referenceConflicts == null) { + return Optional.empty(); + } + + List conflicts = + referenceConflicts.conflicts().stream() + .filter(c -> handledConflictTypes.contains(c.conflictType())) + .collect(Collectors.toList()); + if (conflicts.size() != 1) { + return Optional.empty(); + } + + Conflict conflict = conflicts.get(0); + return Optional.of(conflict); + } } 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 fb7f4ea3095c..b6ae90650e89 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java @@ -22,8 +22,8 @@ import java.io.IOException; import java.net.URI; -import java.util.AbstractMap; import java.util.Collections; +import java.util.Map; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -31,6 +31,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; @@ -39,6 +40,9 @@ import org.junit.jupiter.api.Test; import org.projectnessie.client.ext.NessieClientFactory; import org.projectnessie.client.ext.NessieClientUri; +import org.projectnessie.error.NessieConflictException; +import org.projectnessie.error.NessieNotFoundException; +import org.projectnessie.model.Branch; public class TestMultipleClients extends BaseTestIceberg { @@ -67,33 +71,87 @@ public void afterEach() throws Exception { } @Test - public void testListNamespaces() { + public void testListNamespaces() throws NessieConflictException, NessieNotFoundException { + Assertions.assertThat(catalog.listNamespaces()).isEmpty(); + Assertions.assertThat(anotherCatalog.listNamespaces()).isEmpty(); + + // listing a non-existent namespace should return empty + Assertions.assertThat(catalog.listNamespaces(Namespace.of("db1"))).isEmpty(); + Assertions.assertThat(anotherCatalog.listNamespaces(Namespace.of("db1"))).isEmpty(); + catalog.createNamespace(Namespace.of("db1"), Collections.emptyMap()); + Assertions.assertThat(catalog.listNamespaces()).containsExactlyInAnyOrder(Namespace.of("db1")); + Assertions.assertThat(anotherCatalog.listNamespaces()) + .containsExactlyInAnyOrder(Namespace.of("db1")); // another client creates a namespace with the same nessie server anotherCatalog.createNamespace(Namespace.of("db2"), Collections.emptyMap()); - Assertions.assertThat(anotherCatalog.listNamespaces()) - .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); Assertions.assertThat(catalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); + Assertions.assertThat(anotherCatalog.listNamespaces()) + .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> catalog.listNamespaces()) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining( + "Cannot list top-level namespaces: ref '%s' is no longer valid", branch); + Assertions.assertThatThrownBy(() -> anotherCatalog.listNamespaces(Namespace.of("db1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining( + "Cannot list child namespaces from 'db1': ref '%s' is no longer valid", branch); } @Test - public void testLoadNamespaceMetadata() { + public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNotFoundException { + Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: namespace1"); + Assertions.assertThatThrownBy( + () -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: namespace1"); + catalog.createNamespace(Namespace.of("namespace1"), Collections.emptyMap()); + + // both clients should see the namespace because we read the HEAD of the ref Assertions.assertThat(catalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("namespace1")); + Assertions.assertThat(anotherCatalog.listNamespaces()) + .containsExactlyInAnyOrder(Namespace.of("namespace1")); - // another client adds a metadata to the same namespace - anotherCatalog.setProperties(Namespace.of("namespace1"), Collections.singletonMap("k1", "v1")); - AbstractMap.SimpleEntry entry = new AbstractMap.SimpleEntry<>("k1", "v1"); + // the other client should not be able to update the namespace + // because it is still on the old ref hash + Assertions.assertThatThrownBy( + () -> + anotherCatalog.setProperties( + Namespace.of("namespace1"), Collections.singletonMap("k1", "v1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: namespace1"); + // the same client adds a metadata to the namespace: expect success + catalog.setProperties(Namespace.of("namespace1"), Collections.singletonMap("k1", "v1")); + + // load metadata from the same client and another client both should work fine + // because we read the HEAD of the ref Assertions.assertThat(anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) - .containsExactly(entry); - + .containsExactly(Map.entry("k1", "v1")); Assertions.assertThat(catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) - .containsExactly(entry); + .containsExactly(Map.entry("k1", "v1")); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot load namespace 'namespace1': ref '%s' is no longer valid", branch); + Assertions.assertThatThrownBy( + () -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot load namespace 'namespace1': ref '%s' is no longer valid", branch); } @Test diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java index 8267329678de..0b1af9763dae 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java @@ -106,7 +106,7 @@ public void testCreatingAndDroppingNamespaceWithContent() throws NessieNotFoundE Assertions.assertThatThrownBy(() -> catalog.dropNamespace(namespace)) .isInstanceOf(NamespaceNotEmptyException.class) - .hasMessage("Namespace 'test' is not empty. One or more tables exist."); + .hasMessageContaining("Namespace 'test' is not empty"); catalog.dropTable(identifier, true); catalog.dropNamespace(namespace); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index f1d6159d93f1..d4e079ad269c 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -18,14 +18,36 @@ */ package org.apache.iceberg.nessie; +import static org.apache.iceberg.types.Types.NestedField.required; + import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; +import org.assertj.core.data.Index; import org.junit.jupiter.api.Test; import org.projectnessie.error.NessieConflictException; import org.projectnessie.error.NessieNotFoundException; import org.projectnessie.model.Branch; +import org.projectnessie.model.CommitMeta; +import org.projectnessie.model.Content; +import org.projectnessie.model.ContentKey; +import org.projectnessie.model.IcebergTable; +import org.projectnessie.model.LogResponse; +import org.projectnessie.model.Operation; import org.projectnessie.model.Reference; public class TestNessieIcebergClient extends BaseTestIceberg { @@ -80,7 +102,7 @@ public void testWithReferenceAfterRecreatingBranch() client .getApi() .deleteBranch() - .branch((Branch) client.getApi().getReference().refName(branch).get()) + .branch((Branch) api.getReference().refName(branch).get()) .delete(); createBranch(branch); @@ -91,15 +113,443 @@ public void testWithReferenceAfterRecreatingBranch() Assertions.assertThat(client.withReference(branch, null)).isNotEqualTo(client); } + @Test + public void testCreateNamespace() throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + client.createNamespace(Namespace.of("a"), Map.of()); + Assertions.assertThat(client.listNamespaces(Namespace.of("a"))).isNotNull(); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .first() + .extracting(LogResponse.LogEntry::getCommitMeta) + .extracting(CommitMeta::getMessage, CommitMeta::getAuthor, CommitMeta::getProperties) + .containsExactly( + "create namespace a", + "iceberg-user", + ImmutableMap.of( + "application-type", "iceberg", + "app-id", "iceberg-nessie")); + } + + @Test + public void testCreateNamespaceInvalid() throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceInvalidBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + Assertions.assertThatIllegalArgumentException() + .isThrownBy(() -> client.createNamespace(Namespace.empty(), Map.of())) + .withMessageContaining("Creating empty namespaces is not supported"); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "b"), Map.of())) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Cannot create namespace 'a.b': parent namespace 'a' does not exist"); + } + + @Test + public void testCreateNamespaceConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Namespace already exists: a"); + + client.commitTable( + null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Another content object with name 'a.tbl' already exists"); + } + + @Test + public void testCreateNamespaceExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + org.projectnessie.model.Namespace nessieNs = + org.projectnessie.model.Namespace.of(ContentKey.of("a")); + commit(branch, "create namespace a", Operation.Put.of(ContentKey.of("a"), nessieNs)); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Namespace already exists: a"); + + IcebergTable table = IcebergTable.of("file:///tmp/iceberg", 1, 1, 1, 1); + commit(branch, "create table a.tbl2", Operation.Put.of(ContentKey.of("a", "tbl"), table)); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageContaining("Another content object with name 'a.tbl' already exists"); + } + + @Test + public void testCreateNamespaceNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "createNamespaceNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("b"), Map.of())) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot create namespace 'b': ref 'createNamespaceNonExistingRefBranch' is no longer valid"); + } + + @Test + public void testDropNamespace() throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + Namespace parent = Namespace.of("a"); + Namespace child = Namespace.of("a", "b"); + + Assertions.assertThat(client.dropNamespace(parent)).isFalse(); + Assertions.assertThat(client.dropNamespace(child)).isFalse(); + + client.createNamespace(parent, Map.of()); + client.createNamespace(child, Map.of()); + + Assertions.assertThat(client.dropNamespace(child)).isTrue(); + Assertions.assertThat(client.dropNamespace(parent)).isTrue(); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .extracting(LogResponse.LogEntry::getCommitMeta) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("drop namespace a"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }, + Index.atIndex(0)) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("drop namespace a.b"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }, + Index.atIndex(1)); + } + + @Test + public void testDropNamespaceNotEmpty() throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceInvalidBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + client.createNamespace(Namespace.of("a", "b"), Map.of()); + + Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) + .isInstanceOf(NamespaceNotEmptyException.class) + .hasMessageContaining("Namespace 'a' is not empty."); + } + + @Test + public void testDropNamespaceConflict() throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + client.commitTable( + null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); + + Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a", "tbl"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Content object with name 'a.tbl' is not a namespace."); + } + + @Test + public void testDropNamespaceExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + org.projectnessie.model.Namespace original = fetchNamespace(ContentKey.of("a"), branch); + org.projectnessie.model.Namespace updated = + org.projectnessie.model.Namespace.builder() + .from(original) + .properties(Map.of("k1", "v1")) + .build(); + commit(branch, "update namespace a", Operation.Put.of(ContentKey.of("a"), updated)); + + Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot drop namespace 'a': Values of existing and expected content for key 'a' are different."); + } + + @Test + public void testDropNamespaceNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "dropNamespaceNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThat(client.dropNamespace(Namespace.of("a"))).isFalse(); + } + + @Test + public void testSetProperties() throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + Namespace ns = Namespace.of("a"); + client.createNamespace(ns, Map.of("k1", "v1a")); + + Assertions.assertThat(client.setProperties(ns, Map.of("k1", "v1b", "k2", "v2"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)) + .hasSize(2) + .containsEntry("k1", "v1b") + .containsEntry("k2", "v2"); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .first() + .extracting(LogResponse.LogEntry::getCommitMeta) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("update namespace a"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }); + } + + @Test + public void testSetPropertiesExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + Namespace ns = Namespace.of("a"); + client.createNamespace(ns, Map.of("k1", "v1a")); + + ContentKey key = ContentKey.of("a"); + org.projectnessie.model.Namespace original = fetchNamespace(key, branch); + org.projectnessie.model.Namespace updated = + org.projectnessie.model.Namespace.builder() + .from(original) + .properties(Map.of("k1", "v1b", "k2", "v2")) + .build(); + commit(branch, "update namespace a", Operation.Put.of(key, updated)); + + // will generate a conflict and a retry + Assertions.assertThat(client.setProperties(ns, Map.of("k1", "v1c", "k3", "v3"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)) + .hasSize(3) + .containsEntry("k1", "v1c") + .containsEntry("k2", "v2") + .containsEntry("k3", "v3"); + } + + @Test + public void testSetPropertiesNonExistingNs() + throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesNonExistingNsBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + commit(branch, "delete namespace a", Operation.Delete.of(ContentKey.of("a"))); + + Assertions.assertThatThrownBy( + () -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1a"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: a"); + } + + @Test + public void testSetPropertiesNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "setPropertiesNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of()); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot update properties on namespace 'a': ref 'setPropertiesNonExistingRefBranch' is no longer valid"); + } + + @Test + public void testRemoveProperties() throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesBranch"; + createBranch(branch); + Map catalogOptions = + Map.of( + CatalogProperties.USER, "iceberg-user", + CatalogProperties.APP_ID, "iceberg-nessie"); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); + + Namespace ns = Namespace.of("a"); + + client.createNamespace(ns, Map.of("k1", "v1", "k2", "v2")); + + Assertions.assertThat(client.removeProperties(ns, Set.of("k1"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k2"); + + List entries = api.getCommitLog().refName(branch).get().getLogEntries(); + Assertions.assertThat(entries) + .isNotEmpty() + .first() + .extracting(LogResponse.LogEntry::getCommitMeta) + .satisfies( + meta -> { + Assertions.assertThat(meta.getMessage()).contains("update namespace a"); + Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + Assertions.assertThat(meta.getProperties()) + .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") + .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); + }); + } + + @Test + public void testRemovePropertiesExternalConflict() + throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesExternalConflictBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + Namespace ns = Namespace.of("a"); + client.createNamespace(ns, Map.of("k1", "v1")); + + ContentKey key = ContentKey.of("a"); + org.projectnessie.model.Namespace original = fetchNamespace(key, branch); + org.projectnessie.model.Namespace updated = + org.projectnessie.model.Namespace.builder() + .from(original) + .properties(Map.of("k2", "v2", "k3", "v3")) + .build(); + commit(branch, "update namespace a", Operation.Put.of(key, updated)); + + // will generate a conflict and a retry + Assertions.assertThat(client.removeProperties(ns, Set.of("k2"))).isTrue(); + + Assertions.assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k3"); + } + + @Test + public void testRemovePropertiesNonExistingNs() + throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesNonExistingNsBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of("k1", "v1")); + + commit(branch, "delete namespace a", Operation.Delete.of(ContentKey.of("a"))); + + Assertions.assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace does not exist: a"); + } + + @Test + public void testRemovePropertiesNonExistingRef() + throws NessieConflictException, NessieNotFoundException { + String branch = "removePropertiesNonExistingRefBranch"; + createBranch(branch); + NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); + + client.createNamespace(Namespace.of("a"), Map.of("k1", "v1")); + + api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); + + Assertions.assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "Cannot update properties on namespace 'a': ref 'removePropertiesNonExistingRefBranch' is no longer valid"); + } + @Test public void testInvalidClientApiVersion() throws IOException { try (NessieCatalog newCatalog = new NessieCatalog()) { newCatalog.setConf(hadoopConfig); ImmutableMap.Builder options = ImmutableMap.builder().put("client-api-version", "3"); - Assertions.assertThatThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); + Assertions.assertThatIllegalArgumentException() + .isThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) + .withMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); } } + + private void commit(String branch, String message, Operation... operations) + throws NessieNotFoundException, NessieConflictException { + Branch ref = (Branch) api.getReference().refName(branch).get(); + api.commitMultipleOperations() + .branch(ref) + .commitMeta(NessieUtil.buildCommitMetadata(message, Map.of())) + .operations(Arrays.asList(operations)) + .commit(); + } + + private org.projectnessie.model.Namespace fetchNamespace(ContentKey key, String branch) + throws NessieNotFoundException { + Reference reference = api.getReference().refName(branch).get(); + Content content = api.getContent().key(key).reference(reference).get().get(key); + return content.unwrap(org.projectnessie.model.Namespace.class).orElseThrow(); + } + + private static TableMetadata newTableMetadata() { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + return TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), SortOrder.unsorted(), null, Map.of()); + } } diff --git a/open-api/requirements.txt b/open-api/requirements.txt index a9be79a486fb..c7a3b253147e 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.24.2 +datamodel-code-generator==0.25.1 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 5da91a16f901..cc70d6d4cd89 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -239,25 +239,7 @@ class ViewVersion(BaseModel): class BaseUpdate(BaseModel): - action: Literal[ - 'assign-uuid', - 'upgrade-format-version', - 'add-schema', - 'set-current-schema', - 'add-spec', - 'set-default-spec', - 'add-sort-order', - 'set-default-sort-order', - 'add-snapshot', - 'set-snapshot-ref', - 'remove-snapshots', - 'remove-snapshot-ref', - 'set-location', - 'set-properties', - 'remove-properties', - 'add-view-version', - 'set-current-view-version', - ] + action: str class AssignUUIDUpdate(BaseUpdate): @@ -265,14 +247,17 @@ class AssignUUIDUpdate(BaseUpdate): Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned """ + action: Literal['assign-uuid'] uuid: str class UpgradeFormatVersionUpdate(BaseUpdate): + action: Literal['upgrade-format-version'] format_version: int = Field(..., alias='format-version') class SetCurrentSchemaUpdate(BaseUpdate): + action: Literal['set-current-schema'] schema_id: int = Field( ..., alias='schema-id', @@ -281,10 +266,12 @@ class SetCurrentSchemaUpdate(BaseUpdate): class AddPartitionSpecUpdate(BaseUpdate): + action: Literal['add-spec'] spec: PartitionSpec class SetDefaultSpecUpdate(BaseUpdate): + action: Literal['set-default-spec'] spec_id: int = Field( ..., alias='spec-id', @@ -293,10 +280,12 @@ class SetDefaultSpecUpdate(BaseUpdate): class AddSortOrderUpdate(BaseUpdate): + action: Literal['add-sort-order'] sort_order: SortOrder = Field(..., alias='sort-order') class SetDefaultSortOrderUpdate(BaseUpdate): + action: Literal['set-default-sort-order'] sort_order_id: int = Field( ..., alias='sort-order-id', @@ -305,38 +294,47 @@ class SetDefaultSortOrderUpdate(BaseUpdate): class AddSnapshotUpdate(BaseUpdate): + action: Literal['add-snapshot'] snapshot: Snapshot class SetSnapshotRefUpdate(BaseUpdate, SnapshotReference): + action: Literal['set-snapshot-ref'] ref_name: str = Field(..., alias='ref-name') class RemoveSnapshotsUpdate(BaseUpdate): + action: Literal['remove-snapshots'] snapshot_ids: List[int] = Field(..., alias='snapshot-ids') class RemoveSnapshotRefUpdate(BaseUpdate): + action: Literal['remove-snapshot-ref'] ref_name: str = Field(..., alias='ref-name') class SetLocationUpdate(BaseUpdate): + action: Literal['set-location'] location: str class SetPropertiesUpdate(BaseUpdate): + action: Literal['set-properties'] updates: Dict[str, str] class RemovePropertiesUpdate(BaseUpdate): + action: Literal['remove-properties'] removals: List[str] class AddViewVersionUpdate(BaseUpdate): + action: Literal['add-view-version'] view_version: ViewVersion = Field(..., alias='view-version') class SetCurrentViewVersionUpdate(BaseUpdate): + action: Literal['set-current-view-version'] view_version_id: int = Field( ..., alias='view-version-id', @@ -420,6 +418,10 @@ class AssertDefaultSortOrderId(TableRequirement): default_sort_order_id: int = Field(..., alias='default-sort-order-id') +class ViewRequirement(BaseModel): + __root__: Any = Field(..., discriminator='type') + + class RegisterTableRequest(BaseModel): name: str metadata_location: str = Field(..., alias='metadata-location') @@ -614,7 +616,7 @@ class TransformTerm(BaseModel): term: Reference -class ReportMetricsRequest1(CommitReport): +class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -730,6 +732,7 @@ class ViewMetadata(BaseModel): class AddSchemaUpdate(BaseUpdate): + action: Literal['add-schema'] schema_: Schema = Field(..., alias='schema') last_column_id: Optional[int] = Field( None, @@ -822,6 +825,7 @@ class CommitViewRequest(BaseModel): identifier: Optional[TableIdentifier] = Field( None, description='View identifier to update' ) + requirements: Optional[List[ViewRequirement]] = None updates: List[ViewUpdate] @@ -874,8 +878,8 @@ class LoadViewResult(BaseModel): config: Optional[Dict[str, str]] = None -class ReportMetricsRequest2(BaseModel): - __root__: Union[ReportMetricsRequest, ReportMetricsRequest1] +class ReportMetricsRequest(BaseModel): + __root__: Union[ReportMetricsRequest1, ReportMetricsRequest2] class ScanReport(BaseModel): @@ -901,7 +905,7 @@ class Schema(StructType): ) -class ReportMetricsRequest(ScanReport): +class ReportMetricsRequest1(ScanReport): report_type: str = Field(..., alias='report-type') @@ -914,4 +918,4 @@ class ReportMetricsRequest(ScanReport): AddSchemaUpdate.update_forward_refs() CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() -ReportMetricsRequest2.update_forward_refs() +ReportMetricsRequest.update_forward_refs() diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index c965793cb229..0b008ace605d 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2130,216 +2130,269 @@ components: type: string BaseUpdate: + discriminator: + propertyName: action + mapping: + assign-uuid: '#/components/schemas/AssignUUIDUpdate' + upgrade-format-version: '#/components/schemas/UpgradeFormatVersionUpdate' + add-schema: '#/components/schemas/AddSchemaUpdate' + set-current-schema: '#/components/schemas/SetCurrentSchemaUpdate' + add-spec: '#/components/schemas/AddPartitionSpecUpdate' + set-default-spec: '#/components/schemas/SetDefaultSpecUpdate' + add-sort-order: '#/components/schemas/AddSortOrderUpdate' + set-default-sort-order: '#/components/schemas/SetDefaultSortOrderUpdate' + add-snapshot: '#/components/schemas/AddSnapshotUpdate' + set-snapshot-ref: '#/components/schemas/SetSnapshotRefUpdate' + remove-snapshots: '#/components/schemas/RemoveSnapshotsUpdate' + remove-snapshot-ref: '#/components/schemas/RemoveSnapshotRefUpdate' + set-location: '#/components/schemas/SetLocationUpdate' + set-properties: '#/components/schemas/SetPropertiesUpdate' + remove-properties: '#/components/schemas/RemovePropertiesUpdate' + add-view-version: '#/components/schemas/AddViewVersionUpdate' + set-current-view-version: '#/components/schemas/SetCurrentViewVersionUpdate' type: object required: - action properties: action: type: string - enum: - - assign-uuid - - upgrade-format-version - - add-schema - - set-current-schema - - add-spec - - set-default-spec - - add-sort-order - - set-default-sort-order - - add-snapshot - - set-snapshot-ref - - remove-snapshots - - remove-snapshot-ref - - set-location - - set-properties - - remove-properties - - add-view-version - - set-current-view-version AssignUUIDUpdate: description: Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - uuid - properties: - uuid: - type: string + required: + - action + - uuid + properties: + action: + type: string + enum: ["assign-uuid"] + uuid: + type: string UpgradeFormatVersionUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - format-version - properties: - format-version: - type: integer + required: + - action + - format-version + properties: + action: + type: string + enum: ["upgrade-format-version"] + format-version: + type: integer AddSchemaUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - schema - properties: - schema: - $ref: '#/components/schemas/Schema' - last-column-id: - type: integer - description: The highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side. + required: + - action + - schema + properties: + action: + type: string + enum: ["add-schema"] + schema: + $ref: '#/components/schemas/Schema' + last-column-id: + type: integer + description: The highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side. SetCurrentSchemaUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - schema-id - properties: - schema-id: - type: integer - description: Schema ID to set as current, or -1 to set last added schema + required: + - action + - schema-id + properties: + action: + type: string + enum: ["set-current-schema"] + schema-id: + type: integer + description: Schema ID to set as current, or -1 to set last added schema AddPartitionSpecUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - spec - properties: - spec: - $ref: '#/components/schemas/PartitionSpec' + required: + - action + - spec + properties: + action: + type: string + enum: ["add-spec"] + spec: + $ref: '#/components/schemas/PartitionSpec' SetDefaultSpecUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - spec-id - properties: - spec-id: - type: integer - description: Partition spec ID to set as the default, or -1 to set last added spec + required: + - action + - spec-id + properties: + action: + type: string + enum: [ "set-default-spec" ] + spec-id: + type: integer + description: Partition spec ID to set as the default, or -1 to set last added spec AddSortOrderUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - sort-order - properties: - sort-order: - $ref: '#/components/schemas/SortOrder' + required: + - action + - sort-order + properties: + action: + type: string + enum: [ "add-sort-order" ] + sort-order: + $ref: '#/components/schemas/SortOrder' SetDefaultSortOrderUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - sort-order-id - properties: - sort-order-id: - type: integer - description: Sort order ID to set as the default, or -1 to set last added sort order + required: + - action + - sort-order-id + properties: + action: + type: string + enum: [ "set-default-sort-order" ] + sort-order-id: + type: integer + description: Sort order ID to set as the default, or -1 to set last added sort order AddSnapshotUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - snapshot - properties: - snapshot: - $ref: '#/components/schemas/Snapshot' + required: + - action + - snapshot + properties: + action: + type: string + enum: [ "add-snapshot" ] + snapshot: + $ref: '#/components/schemas/Snapshot' SetSnapshotRefUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - $ref: '#/components/schemas/SnapshotReference' - - type: object - required: - - ref-name - properties: - ref-name: - type: string + required: + - action + - ref-name + properties: + action: + type: string + enum: [ "set-snapshot-ref" ] + ref-name: + type: string RemoveSnapshotsUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - snapshot-ids - properties: - snapshot-ids: - type: array - items: - type: integer - format: int64 + required: + - action + - snapshot-ids + properties: + action: + type: string + enum: [ "remove-snapshots" ] + snapshot-ids: + type: array + items: + type: integer + format: int64 RemoveSnapshotRefUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - ref-name - properties: - ref-name: - type: string + required: + - action + - ref-name + properties: + action: + type: string + enum: [ "remove-snapshot-ref" ] + ref-name: + type: string SetLocationUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - location - properties: - location: - type: string + required: + - action + - location + properties: + action: + type: string + enum: [ "set-location" ] + location: + type: string SetPropertiesUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - updates - properties: - updates: - type: object - additionalProperties: - type: string + required: + - action + - updates + properties: + action: + type: string + enum: [ "set-properties" ] + updates: + type: object + additionalProperties: + type: string RemovePropertiesUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - removals - properties: - removals: - type: array - items: - type: string + required: + - action + - removals + properties: + action: + type: string + enum: [ "remove-properties" ] + removals: + type: array + items: + type: string AddViewVersionUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - view-version - properties: - view-version: - $ref: '#/components/schemas/ViewVersion' + required: + - action + - view-version + properties: + action: + type: string + enum: [ "add-view-version" ] + view-version: + $ref: '#/components/schemas/ViewVersion' SetCurrentViewVersionUpdate: allOf: - $ref: '#/components/schemas/BaseUpdate' - - type: object - required: - - view-version-id - properties: - view-version-id: - type: integer - description: The view version id to set as current, or -1 to set last added view version id + required: + - action + - view-version-id + properties: + action: + type: string + enum: [ "set-current-view-version" ] + view-version-id: + type: integer + description: The view version id to set as current, or -1 to set last added view version id TableUpdate: anyOf: @@ -2510,6 +2563,32 @@ components: default-sort-order-id: type: integer + ViewRequirement: + discriminator: + propertyName: type + mapping: + assert-view-uuid: '#/components/schemas/AssertViewUUID' + type: object + required: + - type + properties: + type: + type: "string" + + AssertViewUUID: + allOf: + - $ref: "#/components/schemas/ViewRequirement" + description: The view UUID must match the requirement's `uuid` + required: + - type + - uuid + properties: + type: + type: string + enum: [ "assert-view-uuid" ] + uuid: + type: string + LoadTableResult: description: | Result used when a table is successfully loaded. @@ -2576,6 +2655,10 @@ components: identifier: description: View identifier to update $ref: '#/components/schemas/TableIdentifier' + requirements: + type: array + items: + $ref: '#/components/schemas/ViewRequirement' updates: type: array items: diff --git a/settings.gradle b/settings.gradle index 9ab130413077..a487fe1ffdac 100644 --- a/settings.gradle +++ b/settings.gradle @@ -106,15 +106,6 @@ if (!flinkVersions.isEmpty()) { project(':flink').name = 'iceberg-flink' } -if (flinkVersions.contains("1.15")) { - include ":iceberg-flink:flink-1.15" - include ":iceberg-flink:flink-runtime-1.15" - project(":iceberg-flink:flink-1.15").projectDir = file('flink/v1.15/flink') - project(":iceberg-flink:flink-1.15").name = "iceberg-flink-1.15" - project(":iceberg-flink:flink-runtime-1.15").projectDir = file('flink/v1.15/flink-runtime') - project(":iceberg-flink:flink-runtime-1.15").name = "iceberg-flink-runtime-1.15" -} - if (flinkVersions.contains("1.16")) { include ":iceberg-flink:flink-1.16" include ":iceberg-flink:flink-runtime-1.16" @@ -133,6 +124,15 @@ if (flinkVersions.contains("1.17")) { 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" + project(":iceberg-flink:flink-1.18").projectDir = file('flink/v1.18/flink') + project(":iceberg-flink:flink-1.18").name = "iceberg-flink-1.18" + project(":iceberg-flink:flink-runtime-1.18").projectDir = file('flink/v1.18/flink-runtime') + project(":iceberg-flink:flink-runtime-1.18").name = "iceberg-flink-runtime-1.18" +} + if (sparkVersions.contains("3.2")) { include ":iceberg-spark:spark-3.2_${scalaVersion}" include ":iceberg-spark:spark-extensions-3.2_${scalaVersion}" diff --git a/site/requirements.txt b/site/requirements.txt index 33b14fe4f540..5ae04767048e 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.4.12 +mkdocs-material==9.5.1 mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin==1.0.5 mkdocs-redirects==1.2.1 diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 40625b5e3450..2675c1010baa 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -339,4 +339,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 40625b5e3450..2675c1010baa 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -339,4 +339,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 02bbec6824f3..48decf995fbb 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -171,7 +171,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 574d014e8335..eddcdb1819ee 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index e08bc4574dbf..1368c26792ee 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -426,6 +426,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { }); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index db8ffe07b8b7..63396e0969d6 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -77,7 +77,7 @@ public void setupTempDirs() { @After public void dropTables() { - sql("DROP TABLE IF EXISTS %s", sourceTableName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceTableName); sql("DROP TABLE IF EXISTS %s", tableName); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index dc22fc4b704c..8e64eaad44e7 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -331,4 +331,39 @@ public void testReplacePartitionField() { row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 6e2bf99c5422..367d95d8f619 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -46,7 +46,7 @@ public TestSnapshotTableProcedure( @After public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %S", sourceName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceName); } @Test diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index d6fab897d62d..25d7e7471588 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -52,7 +52,7 @@ public void testGettingStarted() throws IOException { Assert.assertEquals( "Should have inserted 3 rows", 3L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - sql("DROP TABLE IF EXISTS source"); + sql("DROP TABLE IF EXISTS source PURGE"); sql( "CREATE TABLE source (id bigint, data string) USING parquet LOCATION '%s'", temp.newFolder()); @@ -62,7 +62,7 @@ public void testGettingStarted() throws IOException { Assert.assertEquals( "Table should now have 4 rows", 4L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - sql("DROP TABLE IF EXISTS updates"); + sql("DROP TABLE IF EXISTS updates PURGE"); sql( "CREATE TABLE updates (id bigint, data string) USING parquet LOCATION '%s'", temp.newFolder()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 6958ebc1034a..6b7becc77cd7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -170,7 +170,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index e200bee03e9f..bbc7434138ed 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index dacaee7d8030..161c2e0ba637 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -421,6 +421,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 58edd4145602..2c58281904c9 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -158,6 +158,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation project(path: ':iceberg-parquet') testImplementation project(path: ':iceberg-hive-metastore') testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') testImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') diff --git a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java index aa310d8f7fc1..73bbbd85235d 100644 --- a/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java +++ b/spark/v3.5/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg; -import static org.apache.spark.sql.functions.lit; - import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; import java.io.IOException; @@ -29,25 +27,15 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.Spark3Util; -import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkSessionCatalog; -import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; import org.apache.iceberg.util.ThreadPools; -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.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; @@ -83,10 +71,8 @@ public class DeleteFileIndexBenchmark { private static final String PARTITION_COLUMN = "ss_ticket_number"; private static final int NUM_PARTITIONS = 50; - 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 = 100; - private static final int NUM_ROWS_PER_DATA_FILE = 500; private final Configuration hadoopConf = new Configuration(); private SparkSession spark; @@ -148,85 +134,26 @@ private DeleteFileIndex buildDeletes() { .build(); } - private DataFile loadAddedDataFile() { - table.refresh(); - - Iterable addedDataFiles = table.currentSnapshot().addedDataFiles(table.io()); - return Iterables.getOnlyElement(addedDataFiles); - } - - private DeleteFile loadAddedDeleteFile() { - table.refresh(); - - Iterable addedDeleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); - return Iterables.getOnlyElement(addedDeleteFiles); - } - - private void initDataAndDeletes() throws NoSuchTableException { - Schema schema = table.schema(); - PartitionSpec spec = table.spec(); - LocationProvider locations = table.locationProvider(); - + private void initDataAndDeletes() { for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { - Dataset inputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); - - 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 = FileGenerationUtil.generateDataFile(table, partition); + rowDelta.addRows(dataFile); } - append.commit(); - - RowDelta rowDelta = table.newRowDelta(); - 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(); } } - private void appendAsFile(Dataset df) throws NoSuchTableException { - df.coalesce(1).writeTo(TABLE_NAME).append(); - } - - private Dataset randomDataDF(Schema schema, int numRows) { - Iterable rows = RandomData.generateSpark(schema, numRows, 0); - JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD rowRDD = context.parallelize(Lists.newArrayList(rows)); - StructType rowSparkType = SparkSchemaUtil.convert(schema); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); - } - private void setupSpark() { this.spark = SparkSession.builder() diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index dc22fc4b704c..b230a6912984 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -316,19 +316,61 @@ public void testReplacePartitionField() { sql( "INSERT INTO %s VALUES (1, CAST('2022-01-01 10:00:00' AS TIMESTAMP), CAST('2022-01-01' AS DATE))", tableName); + sql( + "INSERT INTO %s VALUES (2, CAST('2022-01-01 11:00:00' AS TIMESTAMP), CAST('2022-01-01' AS DATE))", + tableName); assertEquals( "Should have expected rows", ImmutableList.of( - row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), - sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); + row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01")), + row(2, Timestamp.valueOf("2022-01-01 11:00:00"), Date.valueOf("2022-01-01"))), + sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); - sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableName); + List output = + sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableName); + assertEquals("Procedure output must match", ImmutableList.of(row(2, 1)), output); assertEquals( "Should have expected rows", ImmutableList.of( - row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01"))), - sql("SELECT * FROM %s WHERE ts < current_timestamp()", tableName)); + row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01")), + row(2, Timestamp.valueOf("2022-01-01 11:00:00"), Date.valueOf("2022-01-01"))), + sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); + } + + @Test + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 467b42899480..eef0f0703bc3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -170,7 +170,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java index c8becc7e5a0f..e59077ae3da9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -46,7 +46,8 @@ class RewriteManifestsProcedure extends BaseProcedure { private static final ProcedureParameter[] PARAMETERS = new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), - ProcedureParameter.optional("use_caching", DataTypes.BooleanType) + ProcedureParameter.optional("use_caching", DataTypes.BooleanType), + ProcedureParameter.optional("spec_id", DataTypes.IntegerType) }; // counts are not nullable since the action result is never null @@ -85,6 +86,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Boolean useCaching = args.isNullAt(1) ? null : args.getBoolean(1); + Integer specId = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable( tableIdent, @@ -95,6 +97,10 @@ public InternalRow[] call(InternalRow args) { action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); } + if (specId != null) { + action.specId(specId); + } + RewriteManifests.Result result = action.execute(); return toOutputRows(result); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index e200bee03e9f..bbc7434138ed 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index dacaee7d8030..161c2e0ba637 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -421,6 +421,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write